diff --git a/airbyte-cdk/python/airbyte_cdk/models/airbyte_protocol.py b/airbyte-cdk/python/airbyte_cdk/models/airbyte_protocol.py index 477cfb8a66fd..6c0cdbb1bac5 100644 --- a/airbyte-cdk/python/airbyte_cdk/models/airbyte_protocol.py +++ b/airbyte-cdk/python/airbyte_cdk/models/airbyte_protocol.py @@ -3,8 +3,9 @@ # from dataclasses import InitVar, dataclass -from typing import Annotated, Any, Dict, List, Mapping, Optional +from typing import Annotated, Any, Dict, List, Mapping, Optional, Union +from airbyte_cdk.models.file_transfer_record_message import AirbyteFileTransferRecordMessage from airbyte_protocol_dataclasses.models import * from serpyco_rs.metadata import Alias @@ -76,7 +77,7 @@ class AirbyteMessage: spec: Optional[ConnectorSpecification] = None # type: ignore [name-defined] connectionStatus: Optional[AirbyteConnectionStatus] = None # type: ignore [name-defined] catalog: Optional[AirbyteCatalog] = None # type: ignore [name-defined] - record: Optional[AirbyteRecordMessage] = None # type: ignore [name-defined] + record: Optional[Union[AirbyteFileTransferRecordMessage, AirbyteRecordMessage]] = None # type: ignore [name-defined] state: Optional[AirbyteStateMessage] = None trace: Optional[AirbyteTraceMessage] = None # type: ignore [name-defined] control: Optional[AirbyteControlMessage] = None # type: ignore [name-defined] diff --git a/airbyte-cdk/python/airbyte_cdk/models/file_transfer_record_message.py b/airbyte-cdk/python/airbyte_cdk/models/file_transfer_record_message.py new file mode 100644 index 000000000000..dcc1b7a92cf1 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/models/file_transfer_record_message.py @@ -0,0 +1,13 @@ +# Copyright (c) 2024 Airbyte, Inc., all rights reserved. + +from dataclasses import dataclass +from typing import Any, Dict, Optional + + +@dataclass +class AirbyteFileTransferRecordMessage: + stream: str + file: Dict[str, Any] + emitted_at: int + namespace: Optional[str] = None + data: Optional[Dict[str, Any]] = None diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/config/abstract_file_based_spec.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/config/abstract_file_based_spec.py index 862c987000bc..38159698816c 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/config/abstract_file_based_spec.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/config/abstract_file_based_spec.py @@ -4,14 +4,33 @@ import copy from abc import abstractmethod -from typing import Any, Dict, List, Optional +from typing import Any, Dict, List, Literal, Optional, Union import dpath +from airbyte_cdk import OneOfOptionConfig from airbyte_cdk.sources.file_based.config.file_based_stream_config import FileBasedStreamConfig from airbyte_cdk.sources.utils import schema_helpers from pydantic.v1 import AnyUrl, BaseModel, Field +class DeliverRecords(BaseModel): + class Config(OneOfOptionConfig): + title = "Replicate Records" + description = "Recommended - Extract and load structured records into your destination of choice. This is the classic method of moving data in Airbyte. It allows for blocking and hashing individual fields or files from a structured schema. Data can be flattened, typed and deduped depending on the destination." + discriminator = "delivery_type" + + delivery_type: Literal["use_records_transfer"] = Field("use_records_transfer", const=True) + + +class DeliverRawFiles(BaseModel): + class Config(OneOfOptionConfig): + title = "Copy Raw Files" + description = "Copy raw files without parsing their contents. Bits are copied into the destination exactly as they appeared in the source. Recommended for use with unstructured text data, non-text and compressed files." + discriminator = "delivery_type" + + delivery_type: Literal["use_file_transfer"] = Field("use_file_transfer", const=True) + + class AbstractFileBasedSpec(BaseModel): """ Used during spec; allows the developer to configure the cloud provider specific options @@ -34,6 +53,17 @@ class AbstractFileBasedSpec(BaseModel): order=10, ) + delivery_method: Union[DeliverRecords, DeliverRawFiles] = Field( + title="Delivery Method", + discriminator="delivery_type", + type="object", + order=7, + display_type="radio", + group="advanced", + default="use_records_transfer", + airbyte_hidden=True, + ) + @classmethod @abstractmethod def documentation_url(cls) -> AnyUrl: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/exceptions.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/exceptions.py index 5bb43e43fbd3..60adf3214e79 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/exceptions.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/exceptions.py @@ -121,3 +121,7 @@ class CustomFileBasedException(AirbyteTracedException): """ pass + + +class FileSizeLimitError(CustomFileBasedException): + pass diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_source.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_source.py index b023287598bb..edd292f42a93 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_source.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_source.py @@ -6,7 +6,7 @@ import traceback from abc import ABC from collections import Counter -from typing import Any, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Type, Union +from typing import Any, Iterator, List, Mapping, Optional, Tuple, Type, Union from airbyte_cdk.logger import AirbyteLogFormatter, init_logger from airbyte_cdk.models import ( @@ -127,10 +127,16 @@ def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> if not isinstance(stream, AbstractFileBasedStream): raise ValueError(f"Stream {stream} is not a file-based stream.") try: + parsed_config = self._get_parsed_config(config) + availability_method = ( + stream.availability_strategy.check_availability + if self._use_file_transfer(parsed_config) + else stream.availability_strategy.check_availability_and_parsability + ) ( stream_is_available, reason, - ) = stream.availability_strategy.check_availability_and_parsability(stream, logger, self) + ) = availability_method(stream, logger, self) except AirbyteTracedException as ate: errors.append(f"Unable to connect to stream {stream.name} - {ate.message}") tracebacks.append(traceback.format_exc()) @@ -217,11 +223,19 @@ def streams(self, config: Mapping[str, Any]) -> List[Stream]: CursorField(DefaultFileBasedStream.ab_last_mod_col), ) stream = FileBasedStreamFacade.create_from_stream( - self._make_default_stream(stream_config, cursor), self, self.logger, stream_state, cursor + stream=self._make_default_stream( + stream_config=stream_config, cursor=cursor, use_file_transfer=self._use_file_transfer(parsed_config) + ), + source=self, + logger=self.logger, + state=stream_state, + cursor=cursor, ) else: cursor = self.cursor_cls(stream_config) - stream = self._make_default_stream(stream_config, cursor) + stream = self._make_default_stream( + stream_config=stream_config, cursor=cursor, use_file_transfer=self._use_file_transfer(parsed_config) + ) streams.append(stream) return streams @@ -230,7 +244,7 @@ def streams(self, config: Mapping[str, Any]) -> List[Stream]: raise ConfigValidationError(FileBasedSourceError.CONFIG_VALIDATION_ERROR) from exc def _make_default_stream( - self, stream_config: FileBasedStreamConfig, cursor: Optional[AbstractFileBasedCursor] + self, stream_config: FileBasedStreamConfig, cursor: Optional[AbstractFileBasedCursor], use_file_transfer: bool = False ) -> AbstractFileBasedStream: return DefaultFileBasedStream( config=stream_config, @@ -242,6 +256,7 @@ def _make_default_stream( validation_policy=self._validate_and_get_validation_policy(stream_config), errors_collector=self.errors_collector, cursor=cursor, + use_file_transfer=use_file_transfer, ) def _get_stream_from_catalog(self, stream_config: FileBasedStreamConfig) -> Optional[AirbyteStream]: @@ -264,7 +279,7 @@ def read( logger: logging.Logger, config: Mapping[str, Any], catalog: ConfiguredAirbyteCatalog, - state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + state: Optional[List[AirbyteStateMessage]] = None, ) -> Iterator[AirbyteMessage]: yield from super().read(logger, config, catalog, state) # emit all the errors collected @@ -298,3 +313,10 @@ def _validate_and_get_validation_policy(self, stream_config: FileBasedStreamConf def _validate_input_schema(self, stream_config: FileBasedStreamConfig) -> None: if stream_config.schemaless and stream_config.input_schema: raise ValidationError("`input_schema` and `schemaless` options cannot both be set", model=FileBasedStreamConfig) + + @staticmethod + def _use_file_transfer(parsed_config: AbstractFileBasedSpec) -> bool: + use_file_transfer = ( + hasattr(parsed_config.delivery_method, "delivery_type") and parsed_config.delivery_method.delivery_type == "use_file_transfer" + ) + return use_file_transfer diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_stream_reader.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_stream_reader.py index 4a7de3bb6992..cb3d0b1920fc 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_stream_reader.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_based_stream_reader.py @@ -92,6 +92,16 @@ def filter_files_by_globs_and_start_date(self, files: List[RemoteFile], globs: L seen.add(file.uri) yield file + @abstractmethod + def file_size(self, file: RemoteFile) -> int: + """Utility method to get size of the remote file. + + This is required for connectors that will support writing to + files. If the connector does not support writing files, then the + subclass can simply `return 0`. + """ + ... + @staticmethod def file_matches_globs(file: RemoteFile, globs: List[str]) -> bool: # Use the GLOBSTAR flag to enable recursive ** matching @@ -105,3 +115,11 @@ def get_prefixes_from_globs(globs: List[str]) -> Set[str]: """ prefixes = {glob.split("*")[0] for glob in globs} return set(filter(lambda x: bool(x), prefixes)) + + def use_file_transfer(self) -> bool: + if self.config: + use_file_transfer = ( + hasattr(self.config.delivery_method, "delivery_type") and self.config.delivery_method.delivery_type == "use_file_transfer" + ) + return use_file_transfer + return False diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/__init__.py index 0faabf0e97f9..5b3c06d26dab 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/__init__.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/__init__.py @@ -14,6 +14,7 @@ from .jsonl_parser import JsonlParser from .parquet_parser import ParquetParser from .unstructured_parser import UnstructuredParser +from .blob_transfer import BlobTransfer default_parsers: Mapping[Type[Any], FileTypeParser] = { AvroFormat: AvroParser(), @@ -24,4 +25,4 @@ UnstructuredFormat: UnstructuredParser(), } -__all__ = ["AvroParser", "CsvParser", "ExcelParser", "JsonlParser", "ParquetParser", "UnstructuredParser", "default_parsers"] +__all__ = ["AvroParser", "CsvParser", "ExcelParser", "JsonlParser", "ParquetParser", "UnstructuredParser", "BlobTransfer", "default_parsers"] diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/blob_transfer.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/blob_transfer.py new file mode 100644 index 000000000000..411335f5845e --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/file_types/blob_transfer.py @@ -0,0 +1,59 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# +import logging +from io import IOBase +from typing import Any, Dict, Generator, Iterable, Optional, Tuple + +from airbyte_cdk.sources.file_based.config.file_based_stream_config import FileBasedStreamConfig +from airbyte_cdk.sources.file_based.file_based_stream_reader import AbstractFileBasedStreamReader, FileReadMode +from airbyte_cdk.sources.file_based.remote_file import RemoteFile +from airbyte_cdk.sources.file_based.writers.local_file_client import LocalFileTransferClient + + +class _FileReader: + def read_data( + self, + config: FileBasedStreamConfig, + file: RemoteFile, + stream_reader: AbstractFileBasedStreamReader, + logger: logging.Logger, + file_read_mode: FileReadMode, + ) -> Generator[Tuple[IOBase, int], None, None]: + + try: + file_size = stream_reader.file_size(file) + with stream_reader.open_file(file, file_read_mode, "UTF-8", logger) as fp: + yield fp, file_size + + except Exception as ex: + logger.error("An error has occurred while reading file: %s", str(ex)) + + +class BlobTransfer: + def __init__(self, file_reader: Optional[_FileReader] = None): + self._file_reader = file_reader if file_reader else _FileReader() + + def write_streams( + self, + config: FileBasedStreamConfig, + file: RemoteFile, + stream_reader: AbstractFileBasedStreamReader, + logger: logging.Logger, + ) -> Iterable[Dict[str, Any]]: + file_no = 0 + try: + data_generator = self._file_reader.read_data(config, file, stream_reader, logger, self.file_read_mode) + local_writer = LocalFileTransferClient() + for file_opened, file_size in data_generator: + yield local_writer.write(file.uri, file_opened, file_size, logger) + file_no += 1 + except Exception as ex: + logger.error("An error has occurred while writing file: %s", str(ex)) + raise ex + finally: + data_generator.close() + + @property + def file_read_mode(self) -> FileReadMode: + return FileReadMode.READ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/schema_helpers.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/schema_helpers.py index c7c7bfa32288..fb7141201d79 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/schema_helpers.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/schema_helpers.py @@ -14,6 +14,7 @@ SchemaType = Mapping[str, Mapping[str, JsonSchemaSupportedType]] schemaless_schema = {"type": "object", "properties": {"data": {"type": "object"}}} +file_transfer_schema = {"type": "object", "properties": {"data": {"type": "object"}, "file": {"type": "object"}}} @total_ordering diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/stream/default_file_based_stream.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/stream/default_file_based_stream.py index 77747c3cbb58..d470c8406e80 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/file_based/stream/default_file_based_stream.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/stream/default_file_based_stream.py @@ -7,7 +7,7 @@ import traceback from copy import deepcopy from functools import cache -from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Set, Union +from typing import Any, Dict, Iterable, List, Mapping, MutableMapping, Optional, Set, Union from airbyte_cdk.models import AirbyteLogMessage, AirbyteMessage, FailureType, Level from airbyte_cdk.models import Type as MessageType @@ -20,8 +20,9 @@ SchemaInferenceError, StopSyncPerValidationPolicy, ) +from airbyte_cdk.sources.file_based.file_types import BlobTransfer from airbyte_cdk.sources.file_based.remote_file import RemoteFile -from airbyte_cdk.sources.file_based.schema_helpers import SchemaType, merge_schemas, schemaless_schema +from airbyte_cdk.sources.file_based.schema_helpers import SchemaType, file_transfer_schema, merge_schemas, schemaless_schema from airbyte_cdk.sources.file_based.stream import AbstractFileBasedStream from airbyte_cdk.sources.file_based.stream.cursor import AbstractFileBasedCursor from airbyte_cdk.sources.file_based.types import StreamSlice @@ -37,12 +38,18 @@ class DefaultFileBasedStream(AbstractFileBasedStream, IncrementalMixin): The default file-based stream. """ + FILE_TRANSFER_KW = "use_file_transfer" DATE_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ" ab_last_mod_col = "_ab_source_file_last_modified" ab_file_name_col = "_ab_source_file_url" + modified = "modified" + source_file_url = "source_file_url" airbyte_columns = [ab_last_mod_col, ab_file_name_col] + use_file_transfer = False def __init__(self, **kwargs: Any): + if self.FILE_TRANSFER_KW in kwargs: + self.use_file_transfer = kwargs.pop(self.FILE_TRANSFER_KW, False) super().__init__(**kwargs) @property @@ -68,6 +75,15 @@ def cursor(self, value: AbstractFileBasedCursor) -> None: def primary_key(self) -> PrimaryKeyType: return self.config.primary_key or self.get_parser().get_parser_defined_primary_key(self.config) + def _filter_schema_invalid_properties(self, configured_catalog_json_schema: Dict[str, Any]) -> Dict[str, Any]: + if self.use_file_transfer: + return { + "type": "object", + "properties": {"file_path": {"type": "string"}, "file_size": {"type": "string"}, self.ab_file_name_col: {"type": "string"}}, + } + else: + return super()._filter_schema_invalid_properties(configured_catalog_json_schema) + def compute_slices(self) -> Iterable[Optional[Mapping[str, Any]]]: # Sort files by last_modified, uri and return them grouped by last_modified all_files = self.list_files() @@ -82,6 +98,12 @@ def transform_record(self, record: dict[str, Any], file: RemoteFile, last_update record[self.ab_file_name_col] = file.uri return record + def transform_record_for_file_transfer(self, record: dict[str, Any], file: RemoteFile) -> dict[str, Any]: + # timstamp() returns a float representing the number of seconds since the unix epoch + record[self.modified] = int(file.last_modified.timestamp()) * 1000 + record[self.source_file_url] = file.uri + return record + def read_records_from_slice(self, stream_slice: StreamSlice) -> Iterable[AirbyteMessage]: """ Yield all records from all remote files in `list_files_for_this_sync`. @@ -101,15 +123,27 @@ def read_records_from_slice(self, stream_slice: StreamSlice) -> Iterable[Airbyte n_skipped = line_no = 0 try: - for record in parser.parse_records(self.config, file, self.stream_reader, self.logger, schema): - line_no += 1 - if self.config.schemaless: - record = {"data": record} - elif not self.record_passes_validation_policy(record): - n_skipped += 1 - continue - record = self.transform_record(record, file, file_datetime_string) - yield stream_data_to_airbyte_message(self.name, record) + if self.use_file_transfer: + self.logger.info(f"{self.name}: {file} file-based syncing") + # todo: complete here the code to not rely on local parser + blob_transfer = BlobTransfer() + for record in blob_transfer.write_streams(self.config, file, self.stream_reader, self.logger): + line_no += 1 + if not self.record_passes_validation_policy(record): + n_skipped += 1 + continue + record = self.transform_record_for_file_transfer(record, file) + yield stream_data_to_airbyte_message(self.name, record, is_file_transfer_message=True) + else: + for record in parser.parse_records(self.config, file, self.stream_reader, self.logger, schema): + line_no += 1 + if self.config.schemaless: + record = {"data": record} + elif not self.record_passes_validation_policy(record): + n_skipped += 1 + continue + record = self.transform_record(record, file, file_datetime_string) + yield stream_data_to_airbyte_message(self.name, record) self._cursor.add_file(file) except StopSyncPerValidationPolicy: @@ -191,7 +225,9 @@ def get_json_schema(self) -> JsonSchema: return {"type": "object", "properties": {**extra_fields, **schema["properties"]}} def _get_raw_json_schema(self) -> JsonSchema: - if self.config.input_schema: + if self.use_file_transfer: + return file_transfer_schema + elif self.config.input_schema: return self.config.get_input_schema() # type: ignore elif self.config.schemaless: return schemaless_schema diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/writers/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/writers/__init__.py new file mode 100644 index 000000000000..8b137891791f --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/writers/__init__.py @@ -0,0 +1 @@ + diff --git a/airbyte-cdk/python/airbyte_cdk/sources/file_based/writers/local_file_client.py b/airbyte-cdk/python/airbyte_cdk/sources/file_based/writers/local_file_client.py new file mode 100644 index 000000000000..1eefbe31c2ee --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/file_based/writers/local_file_client.py @@ -0,0 +1,79 @@ +# Copyright (c) 2024 Airbyte, Inc., all rights reserved. + +import logging +import os +import time +from io import IOBase +from typing import Any, Dict + +import psutil +from airbyte_cdk.models import FailureType +from airbyte_cdk.sources.file_based.exceptions import FileSizeLimitError + +AIRBYTE_STAGING_DIRECTORY = os.getenv("AIRBYTE_STAGING_DIRECTORY", "/staging/files") +DEFAULT_LOCAL_DIRECTORY = "/tmp/airbyte-file-transfer" + + +class LocalFileTransferClient: + FILE_SIZE_LIMIT = 1_000_000_000 + + def __init__(self) -> None: + """ + Initialize the LocalFileTransferClient. It uses a default local directory for file saving. + """ + self._local_directory = AIRBYTE_STAGING_DIRECTORY if os.path.exists(AIRBYTE_STAGING_DIRECTORY) else DEFAULT_LOCAL_DIRECTORY + + def write(self, file_uri: str, fp: IOBase, file_size: int, logger: logging.Logger) -> Dict[str, Any]: + """ + Write the file to a local directory. + """ + if file_size > self.FILE_SIZE_LIMIT: + message = "File size exceeds the 1 GB limit." + raise FileSizeLimitError(message=message, internal_message=message, failure_type=FailureType.config_error) + + # Remove left slashes from source path format to make relative path for writing locally + file_relative_path = file_uri.lstrip("/") + local_file_path = os.path.join(self._local_directory, file_relative_path) + + # Ensure the local directory exists + os.makedirs(os.path.dirname(local_file_path), exist_ok=True) + + # Get the absolute path + absolute_file_path = os.path.abspath(local_file_path) + + # Get available disk space + disk_usage = psutil.disk_usage("/") + available_disk_space = disk_usage.free + + # Get available memory + memory_info = psutil.virtual_memory() + available_memory = memory_info.available + + # logger.info(f"Writing file to {local_file_path}.") + # Log file size, available disk space, and memory + logger.info( + f"Writing file to '{local_file_path}' " + f"with size: {file_size / (1024 * 1024):,.2f} MB ({file_size / (1024 * 1024 * 1024):.2f} GB), " + f"available disk space: {available_disk_space / (1024 * 1024):,.2f} MB ({available_disk_space / (1024 * 1024 * 1024):.2f} GB)," + f"available memory: {available_memory / (1024 * 1024):,.2f} MB ({available_memory / (1024 * 1024 * 1024):.2f} GB)." + ) + + with open(local_file_path, "wb") as f: + # Measure the time for reading + logger.info("Starting to read the file") + start_read_time = time.time() + # todo: read chunks or allow reader to implement their own download process https://github.com/airbytehq/airbyte-internal-issues/issues/10480 + file_content = fp.read() # Read the file content + read_duration = time.time() - start_read_time + logger.info(f"Time taken to read the file: {read_duration:,.2f} seconds.") + + # Measure the time for writing + logger.info("Starting to write the file locally") + start_write_time = time.time() + f.write(file_content) + write_duration = time.time() - start_write_time + logger.info(f"Time taken to write the file: {write_duration:,.2f} seconds.") + + logger.info(f"File {file_relative_path} successfully written to {self._local_directory}.") + + return {"file_url": absolute_file_path, "bytes": file_size, "file_relative_path": file_relative_path} diff --git a/airbyte-cdk/python/airbyte_cdk/sources/utils/record_helper.py b/airbyte-cdk/python/airbyte_cdk/sources/utils/record_helper.py index ec92ac2f9645..98cefd1a8d40 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/utils/record_helper.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/utils/record_helper.py @@ -7,6 +7,7 @@ from airbyte_cdk.models import AirbyteLogMessage, AirbyteMessage, AirbyteRecordMessage, AirbyteTraceMessage from airbyte_cdk.models import Type as MessageType +from airbyte_cdk.models.file_transfer_record_message import AirbyteFileTransferRecordMessage from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.utils.transform import TransformConfig, TypeTransformer @@ -16,6 +17,7 @@ def stream_data_to_airbyte_message( data_or_message: StreamData, transformer: TypeTransformer = TypeTransformer(TransformConfig.NoTransform), schema: Optional[Mapping[str, Any]] = None, + is_file_transfer_message: bool = False, ) -> AirbyteMessage: if schema is None: schema = {} @@ -29,7 +31,10 @@ def stream_data_to_airbyte_message( # taken unless configured. See # docs/connector-development/cdk-python/schemas.md for details. transformer.transform(data, schema) # type: ignore - message = AirbyteRecordMessage(stream=stream_name, data=data, emitted_at=now_millis) + if is_file_transfer_message: + message = AirbyteFileTransferRecordMessage(stream=stream_name, file=data, emitted_at=now_millis, data={}) + else: + message = AirbyteRecordMessage(stream=stream_name, data=data, emitted_at=now_millis) return AirbyteMessage(type=MessageType.RECORD, record=message) case AirbyteTraceMessage(): return AirbyteMessage(type=MessageType.TRACE, trace=data_or_message) diff --git a/airbyte-cdk/python/poetry.lock b/airbyte-cdk/python/poetry.lock index 1019d433060f..be8bfca6febe 100644 --- a/airbyte-cdk/python/poetry.lock +++ b/airbyte-cdk/python/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.6.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.8.3 and should not be changed by hand. [[package]] name = "aiohappyeyeballs" @@ -1192,88 +1192,103 @@ python-dateutil = ">=2.7" [[package]] name = "frozenlist" -version = "1.4.1" +version = "1.5.0" description = "A list-like structure which implements collections.abc.MutableSequence" optional = true python-versions = ">=3.8" files = [ - {file = "frozenlist-1.4.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:f9aa1878d1083b276b0196f2dfbe00c9b7e752475ed3b682025ff20c1c1f51ac"}, - {file = "frozenlist-1.4.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:29acab3f66f0f24674b7dc4736477bcd4bc3ad4b896f5f45379a67bce8b96868"}, - {file = "frozenlist-1.4.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:74fb4bee6880b529a0c6560885fce4dc95936920f9f20f53d99a213f7bf66776"}, - {file = "frozenlist-1.4.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:590344787a90ae57d62511dd7c736ed56b428f04cd8c161fcc5e7232c130c69a"}, - {file = "frozenlist-1.4.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:068b63f23b17df8569b7fdca5517edef76171cf3897eb68beb01341131fbd2ad"}, - {file = "frozenlist-1.4.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5c849d495bf5154cd8da18a9eb15db127d4dba2968d88831aff6f0331ea9bd4c"}, - {file = "frozenlist-1.4.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9750cc7fe1ae3b1611bb8cfc3f9ec11d532244235d75901fb6b8e42ce9229dfe"}, - {file = "frozenlist-1.4.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a9b2de4cf0cdd5bd2dee4c4f63a653c61d2408055ab77b151c1957f221cabf2a"}, - {file = "frozenlist-1.4.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:0633c8d5337cb5c77acbccc6357ac49a1770b8c487e5b3505c57b949b4b82e98"}, - {file = "frozenlist-1.4.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:27657df69e8801be6c3638054e202a135c7f299267f1a55ed3a598934f6c0d75"}, - {file = "frozenlist-1.4.1-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:f9a3ea26252bd92f570600098783d1371354d89d5f6b7dfd87359d669f2109b5"}, - {file = "frozenlist-1.4.1-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:4f57dab5fe3407b6c0c1cc907ac98e8a189f9e418f3b6e54d65a718aaafe3950"}, - {file = "frozenlist-1.4.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:e02a0e11cf6597299b9f3bbd3f93d79217cb90cfd1411aec33848b13f5c656cc"}, - {file = "frozenlist-1.4.1-cp310-cp310-win32.whl", hash = "sha256:a828c57f00f729620a442881cc60e57cfcec6842ba38e1b19fd3e47ac0ff8dc1"}, - {file = "frozenlist-1.4.1-cp310-cp310-win_amd64.whl", hash = "sha256:f56e2333dda1fe0f909e7cc59f021eba0d2307bc6f012a1ccf2beca6ba362439"}, - {file = "frozenlist-1.4.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:a0cb6f11204443f27a1628b0e460f37fb30f624be6051d490fa7d7e26d4af3d0"}, - {file = "frozenlist-1.4.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b46c8ae3a8f1f41a0d2ef350c0b6e65822d80772fe46b653ab6b6274f61d4a49"}, - {file = "frozenlist-1.4.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:fde5bd59ab5357e3853313127f4d3565fc7dad314a74d7b5d43c22c6a5ed2ced"}, - {file = "frozenlist-1.4.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:722e1124aec435320ae01ee3ac7bec11a5d47f25d0ed6328f2273d287bc3abb0"}, - {file = "frozenlist-1.4.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2471c201b70d58a0f0c1f91261542a03d9a5e088ed3dc6c160d614c01649c106"}, - {file = "frozenlist-1.4.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c757a9dd70d72b076d6f68efdbb9bc943665ae954dad2801b874c8c69e185068"}, - {file = "frozenlist-1.4.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f146e0911cb2f1da549fc58fc7bcd2b836a44b79ef871980d605ec392ff6b0d2"}, - {file = "frozenlist-1.4.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f9c515e7914626b2a2e1e311794b4c35720a0be87af52b79ff8e1429fc25f19"}, - {file = "frozenlist-1.4.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:c302220494f5c1ebeb0912ea782bcd5e2f8308037b3c7553fad0e48ebad6ad82"}, - {file = "frozenlist-1.4.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:442acde1e068288a4ba7acfe05f5f343e19fac87bfc96d89eb886b0363e977ec"}, - {file = "frozenlist-1.4.1-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:1b280e6507ea8a4fa0c0a7150b4e526a8d113989e28eaaef946cc77ffd7efc0a"}, - {file = "frozenlist-1.4.1-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:fe1a06da377e3a1062ae5fe0926e12b84eceb8a50b350ddca72dc85015873f74"}, - {file = "frozenlist-1.4.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:db9e724bebd621d9beca794f2a4ff1d26eed5965b004a97f1f1685a173b869c2"}, - {file = "frozenlist-1.4.1-cp311-cp311-win32.whl", hash = "sha256:e774d53b1a477a67838a904131c4b0eef6b3d8a651f8b138b04f748fccfefe17"}, - {file = "frozenlist-1.4.1-cp311-cp311-win_amd64.whl", hash = "sha256:fb3c2db03683b5767dedb5769b8a40ebb47d6f7f45b1b3e3b4b51ec8ad9d9825"}, - {file = "frozenlist-1.4.1-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:1979bc0aeb89b33b588c51c54ab0161791149f2461ea7c7c946d95d5f93b56ae"}, - {file = "frozenlist-1.4.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:cc7b01b3754ea68a62bd77ce6020afaffb44a590c2289089289363472d13aedb"}, - {file = "frozenlist-1.4.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:c9c92be9fd329ac801cc420e08452b70e7aeab94ea4233a4804f0915c14eba9b"}, - {file = "frozenlist-1.4.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c3894db91f5a489fc8fa6a9991820f368f0b3cbdb9cd8849547ccfab3392d86"}, - {file = "frozenlist-1.4.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ba60bb19387e13597fb059f32cd4d59445d7b18b69a745b8f8e5db0346f33480"}, - {file = "frozenlist-1.4.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8aefbba5f69d42246543407ed2461db31006b0f76c4e32dfd6f42215a2c41d09"}, - {file = "frozenlist-1.4.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:780d3a35680ced9ce682fbcf4cb9c2bad3136eeff760ab33707b71db84664e3a"}, - {file = "frozenlist-1.4.1-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9acbb16f06fe7f52f441bb6f413ebae6c37baa6ef9edd49cdd567216da8600cd"}, - {file = "frozenlist-1.4.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:23b701e65c7b36e4bf15546a89279bd4d8675faabc287d06bbcfac7d3c33e1e6"}, - {file = "frozenlist-1.4.1-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:3e0153a805a98f5ada7e09826255ba99fb4f7524bb81bf6b47fb702666484ae1"}, - {file = "frozenlist-1.4.1-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:dd9b1baec094d91bf36ec729445f7769d0d0cf6b64d04d86e45baf89e2b9059b"}, - {file = "frozenlist-1.4.1-cp312-cp312-musllinux_1_1_s390x.whl", hash = "sha256:1a4471094e146b6790f61b98616ab8e44f72661879cc63fa1049d13ef711e71e"}, - {file = "frozenlist-1.4.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:5667ed53d68d91920defdf4035d1cdaa3c3121dc0b113255124bcfada1cfa1b8"}, - {file = "frozenlist-1.4.1-cp312-cp312-win32.whl", hash = "sha256:beee944ae828747fd7cb216a70f120767fc9f4f00bacae8543c14a6831673f89"}, - {file = "frozenlist-1.4.1-cp312-cp312-win_amd64.whl", hash = "sha256:64536573d0a2cb6e625cf309984e2d873979709f2cf22839bf2d61790b448ad5"}, - {file = "frozenlist-1.4.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:20b51fa3f588ff2fe658663db52a41a4f7aa6c04f6201449c6c7c476bd255c0d"}, - {file = "frozenlist-1.4.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:410478a0c562d1a5bcc2f7ea448359fcb050ed48b3c6f6f4f18c313a9bdb1826"}, - {file = "frozenlist-1.4.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:c6321c9efe29975232da3bd0af0ad216800a47e93d763ce64f291917a381b8eb"}, - {file = "frozenlist-1.4.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:48f6a4533887e189dae092f1cf981f2e3885175f7a0f33c91fb5b7b682b6bab6"}, - {file = "frozenlist-1.4.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6eb73fa5426ea69ee0e012fb59cdc76a15b1283d6e32e4f8dc4482ec67d1194d"}, - {file = "frozenlist-1.4.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fbeb989b5cc29e8daf7f976b421c220f1b8c731cbf22b9130d8815418ea45887"}, - {file = "frozenlist-1.4.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:32453c1de775c889eb4e22f1197fe3bdfe457d16476ea407472b9442e6295f7a"}, - {file = "frozenlist-1.4.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:693945278a31f2086d9bf3df0fe8254bbeaef1fe71e1351c3bd730aa7d31c41b"}, - {file = "frozenlist-1.4.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:1d0ce09d36d53bbbe566fe296965b23b961764c0bcf3ce2fa45f463745c04701"}, - {file = "frozenlist-1.4.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:3a670dc61eb0d0eb7080890c13de3066790f9049b47b0de04007090807c776b0"}, - {file = "frozenlist-1.4.1-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:dca69045298ce5c11fd539682cff879cc1e664c245d1c64da929813e54241d11"}, - {file = "frozenlist-1.4.1-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:a06339f38e9ed3a64e4c4e43aec7f59084033647f908e4259d279a52d3757d09"}, - {file = "frozenlist-1.4.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b7f2f9f912dca3934c1baec2e4585a674ef16fe00218d833856408c48d5beee7"}, - {file = "frozenlist-1.4.1-cp38-cp38-win32.whl", hash = "sha256:e7004be74cbb7d9f34553a5ce5fb08be14fb33bc86f332fb71cbe5216362a497"}, - {file = "frozenlist-1.4.1-cp38-cp38-win_amd64.whl", hash = "sha256:5a7d70357e7cee13f470c7883a063aae5fe209a493c57d86eb7f5a6f910fae09"}, - {file = "frozenlist-1.4.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:bfa4a17e17ce9abf47a74ae02f32d014c5e9404b6d9ac7f729e01562bbee601e"}, - {file = "frozenlist-1.4.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b7e3ed87d4138356775346e6845cccbe66cd9e207f3cd11d2f0b9fd13681359d"}, - {file = "frozenlist-1.4.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:c99169d4ff810155ca50b4da3b075cbde79752443117d89429595c2e8e37fed8"}, - {file = "frozenlist-1.4.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:edb678da49d9f72c9f6c609fbe41a5dfb9a9282f9e6a2253d5a91e0fc382d7c0"}, - {file = "frozenlist-1.4.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6db4667b187a6742b33afbbaf05a7bc551ffcf1ced0000a571aedbb4aa42fc7b"}, - {file = "frozenlist-1.4.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:55fdc093b5a3cb41d420884cdaf37a1e74c3c37a31f46e66286d9145d2063bd0"}, - {file = "frozenlist-1.4.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:82e8211d69a4f4bc360ea22cd6555f8e61a1bd211d1d5d39d3d228b48c83a897"}, - {file = "frozenlist-1.4.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:89aa2c2eeb20957be2d950b85974b30a01a762f3308cd02bb15e1ad632e22dc7"}, - {file = "frozenlist-1.4.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:9d3e0c25a2350080e9319724dede4f31f43a6c9779be48021a7f4ebde8b2d742"}, - {file = "frozenlist-1.4.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:7268252af60904bf52c26173cbadc3a071cece75f873705419c8681f24d3edea"}, - {file = "frozenlist-1.4.1-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:0c250a29735d4f15321007fb02865f0e6b6a41a6b88f1f523ca1596ab5f50bd5"}, - {file = "frozenlist-1.4.1-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:96ec70beabbd3b10e8bfe52616a13561e58fe84c0101dd031dc78f250d5128b9"}, - {file = "frozenlist-1.4.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:23b2d7679b73fe0e5a4560b672a39f98dfc6f60df63823b0a9970525325b95f6"}, - {file = "frozenlist-1.4.1-cp39-cp39-win32.whl", hash = "sha256:a7496bfe1da7fb1a4e1cc23bb67c58fab69311cc7d32b5a99c2007b4b2a0e932"}, - {file = "frozenlist-1.4.1-cp39-cp39-win_amd64.whl", hash = "sha256:e6a20a581f9ce92d389a8c7d7c3dd47c81fd5d6e655c8dddf341e14aa48659d0"}, - {file = "frozenlist-1.4.1-py3-none-any.whl", hash = "sha256:04ced3e6a46b4cfffe20f9ae482818e34eba9b5fb0ce4056e4cc9b6e212d09b7"}, - {file = "frozenlist-1.4.1.tar.gz", hash = "sha256:c037a86e8513059a2613aaba4d817bb90b9d9b6b69aace3ce9c877e8c8ed402b"}, + {file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5b6a66c18b5b9dd261ca98dffcb826a525334b2f29e7caa54e182255c5f6a65a"}, + {file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d1b3eb7b05ea246510b43a7e53ed1653e55c2121019a97e60cad7efb881a97bb"}, + {file = "frozenlist-1.5.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:15538c0cbf0e4fa11d1e3a71f823524b0c46299aed6e10ebb4c2089abd8c3bec"}, + {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e79225373c317ff1e35f210dd5f1344ff31066ba8067c307ab60254cd3a78ad5"}, + {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9272fa73ca71266702c4c3e2d4a28553ea03418e591e377a03b8e3659d94fa76"}, + {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:498524025a5b8ba81695761d78c8dd7382ac0b052f34e66939c42df860b8ff17"}, + {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:92b5278ed9d50fe610185ecd23c55d8b307d75ca18e94c0e7de328089ac5dcba"}, + {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7f3c8c1dacd037df16e85227bac13cca58c30da836c6f936ba1df0c05d046d8d"}, + {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:f2ac49a9bedb996086057b75bf93538240538c6d9b38e57c82d51f75a73409d2"}, + {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e66cc454f97053b79c2ab09c17fbe3c825ea6b4de20baf1be28919460dd7877f"}, + {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:5a3ba5f9a0dfed20337d3e966dc359784c9f96503674c2faf015f7fe8e96798c"}, + {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:6321899477db90bdeb9299ac3627a6a53c7399c8cd58d25da094007402b039ab"}, + {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:76e4753701248476e6286f2ef492af900ea67d9706a0155335a40ea21bf3b2f5"}, + {file = "frozenlist-1.5.0-cp310-cp310-win32.whl", hash = "sha256:977701c081c0241d0955c9586ffdd9ce44f7a7795df39b9151cd9a6fd0ce4cfb"}, + {file = "frozenlist-1.5.0-cp310-cp310-win_amd64.whl", hash = "sha256:189f03b53e64144f90990d29a27ec4f7997d91ed3d01b51fa39d2dbe77540fd4"}, + {file = "frozenlist-1.5.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:fd74520371c3c4175142d02a976aee0b4cb4a7cc912a60586ffd8d5929979b30"}, + {file = "frozenlist-1.5.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:2f3f7a0fbc219fb4455264cae4d9f01ad41ae6ee8524500f381de64ffaa077d5"}, + {file = "frozenlist-1.5.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f47c9c9028f55a04ac254346e92977bf0f166c483c74b4232bee19a6697e4778"}, + {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0996c66760924da6e88922756d99b47512a71cfd45215f3570bf1e0b694c206a"}, + {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a2fe128eb4edeabe11896cb6af88fca5346059f6c8d807e3b910069f39157869"}, + {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1a8ea951bbb6cacd492e3948b8da8c502a3f814f5d20935aae74b5df2b19cf3d"}, + {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:de537c11e4aa01d37db0d403b57bd6f0546e71a82347a97c6a9f0dcc532b3a45"}, + {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c2623347b933fcb9095841f1cc5d4ff0b278addd743e0e966cb3d460278840d"}, + {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:cee6798eaf8b1416ef6909b06f7dc04b60755206bddc599f52232606e18179d3"}, + {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:f5f9da7f5dbc00a604fe74aa02ae7c98bcede8a3b8b9666f9f86fc13993bc71a"}, + {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:90646abbc7a5d5c7c19461d2e3eeb76eb0b204919e6ece342feb6032c9325ae9"}, + {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:bdac3c7d9b705d253b2ce370fde941836a5f8b3c5c2b8fd70940a3ea3af7f4f2"}, + {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:03d33c2ddbc1816237a67f66336616416e2bbb6beb306e5f890f2eb22b959cdf"}, + {file = "frozenlist-1.5.0-cp311-cp311-win32.whl", hash = "sha256:237f6b23ee0f44066219dae14c70ae38a63f0440ce6750f868ee08775073f942"}, + {file = "frozenlist-1.5.0-cp311-cp311-win_amd64.whl", hash = "sha256:0cc974cc93d32c42e7b0f6cf242a6bd941c57c61b618e78b6c0a96cb72788c1d"}, + {file = "frozenlist-1.5.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:31115ba75889723431aa9a4e77d5f398f5cf976eea3bdf61749731f62d4a4a21"}, + {file = "frozenlist-1.5.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:7437601c4d89d070eac8323f121fcf25f88674627505334654fd027b091db09d"}, + {file = "frozenlist-1.5.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:7948140d9f8ece1745be806f2bfdf390127cf1a763b925c4a805c603df5e697e"}, + {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:feeb64bc9bcc6b45c6311c9e9b99406660a9c05ca8a5b30d14a78555088b0b3a"}, + {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:683173d371daad49cffb8309779e886e59c2f369430ad28fe715f66d08d4ab1a"}, + {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7d57d8f702221405a9d9b40f9da8ac2e4a1a8b5285aac6100f3393675f0a85ee"}, + {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:30c72000fbcc35b129cb09956836c7d7abf78ab5416595e4857d1cae8d6251a6"}, + {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:000a77d6034fbad9b6bb880f7ec073027908f1b40254b5d6f26210d2dab1240e"}, + {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:5d7f5a50342475962eb18b740f3beecc685a15b52c91f7d975257e13e029eca9"}, + {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:87f724d055eb4785d9be84e9ebf0f24e392ddfad00b3fe036e43f489fafc9039"}, + {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:6e9080bb2fb195a046e5177f10d9d82b8a204c0736a97a153c2466127de87784"}, + {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9b93d7aaa36c966fa42efcaf716e6b3900438632a626fb09c049f6a2f09fc631"}, + {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:52ef692a4bc60a6dd57f507429636c2af8b6046db8b31b18dac02cbc8f507f7f"}, + {file = "frozenlist-1.5.0-cp312-cp312-win32.whl", hash = "sha256:29d94c256679247b33a3dc96cce0f93cbc69c23bf75ff715919332fdbb6a32b8"}, + {file = "frozenlist-1.5.0-cp312-cp312-win_amd64.whl", hash = "sha256:8969190d709e7c48ea386db202d708eb94bdb29207a1f269bab1196ce0dcca1f"}, + {file = "frozenlist-1.5.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:7a1a048f9215c90973402e26c01d1cff8a209e1f1b53f72b95c13db61b00f953"}, + {file = "frozenlist-1.5.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:dd47a5181ce5fcb463b5d9e17ecfdb02b678cca31280639255ce9d0e5aa67af0"}, + {file = "frozenlist-1.5.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:1431d60b36d15cda188ea222033eec8e0eab488f39a272461f2e6d9e1a8e63c2"}, + {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6482a5851f5d72767fbd0e507e80737f9c8646ae7fd303def99bfe813f76cf7f"}, + {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:44c49271a937625619e862baacbd037a7ef86dd1ee215afc298a417ff3270608"}, + {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:12f78f98c2f1c2429d42e6a485f433722b0061d5c0b0139efa64f396efb5886b"}, + {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ce3aa154c452d2467487765e3adc730a8c153af77ad84096bc19ce19a2400840"}, + {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9b7dc0c4338e6b8b091e8faf0db3168a37101943e687f373dce00959583f7439"}, + {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:45e0896250900b5aa25180f9aec243e84e92ac84bd4a74d9ad4138ef3f5c97de"}, + {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:561eb1c9579d495fddb6da8959fd2a1fca2c6d060d4113f5844b433fc02f2641"}, + {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:df6e2f325bfee1f49f81aaac97d2aa757c7646534a06f8f577ce184afe2f0a9e"}, + {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:140228863501b44b809fb39ec56b5d4071f4d0aa6d216c19cbb08b8c5a7eadb9"}, + {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:7707a25d6a77f5d27ea7dc7d1fc608aa0a478193823f88511ef5e6b8a48f9d03"}, + {file = "frozenlist-1.5.0-cp313-cp313-win32.whl", hash = "sha256:31a9ac2b38ab9b5a8933b693db4939764ad3f299fcaa931a3e605bc3460e693c"}, + {file = "frozenlist-1.5.0-cp313-cp313-win_amd64.whl", hash = "sha256:11aabdd62b8b9c4b84081a3c246506d1cddd2dd93ff0ad53ede5defec7886b28"}, + {file = "frozenlist-1.5.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:dd94994fc91a6177bfaafd7d9fd951bc8689b0a98168aa26b5f543868548d3ca"}, + {file = "frozenlist-1.5.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:2d0da8bbec082bf6bf18345b180958775363588678f64998c2b7609e34719b10"}, + {file = "frozenlist-1.5.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:73f2e31ea8dd7df61a359b731716018c2be196e5bb3b74ddba107f694fbd7604"}, + {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:828afae9f17e6de596825cf4228ff28fbdf6065974e5ac1410cecc22f699d2b3"}, + {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f1577515d35ed5649d52ab4319db757bb881ce3b2b796d7283e6634d99ace307"}, + {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2150cc6305a2c2ab33299453e2968611dacb970d2283a14955923062c8d00b10"}, + {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a72b7a6e3cd2725eff67cd64c8f13335ee18fc3c7befc05aed043d24c7b9ccb9"}, + {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c16d2fa63e0800723139137d667e1056bee1a1cf7965153d2d104b62855e9b99"}, + {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:17dcc32fc7bda7ce5875435003220a457bcfa34ab7924a49a1c19f55b6ee185c"}, + {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:97160e245ea33d8609cd2b8fd997c850b56db147a304a262abc2b3be021a9171"}, + {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:f1e6540b7fa044eee0bb5111ada694cf3dc15f2b0347ca125ee9ca984d5e9e6e"}, + {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:91d6c171862df0a6c61479d9724f22efb6109111017c87567cfeb7b5d1449fdf"}, + {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:c1fac3e2ace2eb1052e9f7c7db480818371134410e1f5c55d65e8f3ac6d1407e"}, + {file = "frozenlist-1.5.0-cp38-cp38-win32.whl", hash = "sha256:b97f7b575ab4a8af9b7bc1d2ef7f29d3afee2226bd03ca3875c16451ad5a7723"}, + {file = "frozenlist-1.5.0-cp38-cp38-win_amd64.whl", hash = "sha256:374ca2dabdccad8e2a76d40b1d037f5bd16824933bf7bcea3e59c891fd4a0923"}, + {file = "frozenlist-1.5.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:9bbcdfaf4af7ce002694a4e10a0159d5a8d20056a12b05b45cea944a4953f972"}, + {file = "frozenlist-1.5.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:1893f948bf6681733aaccf36c5232c231e3b5166d607c5fa77773611df6dc336"}, + {file = "frozenlist-1.5.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:2b5e23253bb709ef57a8e95e6ae48daa9ac5f265637529e4ce6b003a37b2621f"}, + {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0f253985bb515ecd89629db13cb58d702035ecd8cfbca7d7a7e29a0e6d39af5f"}, + {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:04a5c6babd5e8fb7d3c871dc8b321166b80e41b637c31a995ed844a6139942b6"}, + {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a9fe0f1c29ba24ba6ff6abf688cb0b7cf1efab6b6aa6adc55441773c252f7411"}, + {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:226d72559fa19babe2ccd920273e767c96a49b9d3d38badd7c91a0fdeda8ea08"}, + {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15b731db116ab3aedec558573c1a5eec78822b32292fe4f2f0345b7f697745c2"}, + {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:366d8f93e3edfe5a918c874702f78faac300209a4d5bf38352b2c1bdc07a766d"}, + {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:1b96af8c582b94d381a1c1f51ffaedeb77c821c690ea5f01da3d70a487dd0a9b"}, + {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:c03eff4a41bd4e38415cbed054bbaff4a075b093e2394b6915dca34a40d1e38b"}, + {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:50cf5e7ee9b98f22bdecbabf3800ae78ddcc26e4a435515fc72d97903e8488e0"}, + {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1e76bfbc72353269c44e0bc2cfe171900fbf7f722ad74c9a7b638052afe6a00c"}, + {file = "frozenlist-1.5.0-cp39-cp39-win32.whl", hash = "sha256:666534d15ba8f0fda3f53969117383d5dc021266b3c1a42c9ec4855e4b58b9d3"}, + {file = "frozenlist-1.5.0-cp39-cp39-win_amd64.whl", hash = "sha256:5c28f4b5dbef8a0d8aad0d4de24d1e9e981728628afaf4ea0792f5d0939372f0"}, + {file = "frozenlist-1.5.0-py3-none-any.whl", hash = "sha256:d994863bba198a4a518b467bb971c56e1db3f180a25c6cf7bb1949c267f748c3"}, + {file = "frozenlist-1.5.0.tar.gz", hash = "sha256:81d5af29e61b9c8348e876d442253723928dce6433e0e76cd925cd83f1b4b817"}, ] [[package]] @@ -1828,13 +1843,13 @@ six = "*" [[package]] name = "langsmith" -version = "0.1.136" +version = "0.1.137" description = "Client library to connect to the LangSmith LLM Tracing and Evaluation Platform." optional = true python-versions = "<4.0,>=3.8.1" files = [ - {file = "langsmith-0.1.136-py3-none-any.whl", hash = "sha256:cad2215eb7a754ee259878e19c558f4f8d3795aa1b699f087d4500e640f80d0a"}, - {file = "langsmith-0.1.136.tar.gz", hash = "sha256:5c0de01a313db70dd9a85845c0f416a69b5b653b3e98ba413d7d41e8851315b1"}, + {file = "langsmith-0.1.137-py3-none-any.whl", hash = "sha256:4256d5c61133749890f7b5c88321dbb133ce0f440c621ea28e76513285859b81"}, + {file = "langsmith-0.1.137.tar.gz", hash = "sha256:56cdfcc6c74cb20a3f437d5bd144feb5bf93f54c5a2918d1e568cbd084a372d4"}, ] [package.dependencies] @@ -3110,6 +3125,36 @@ files = [ {file = "propcache-0.2.0.tar.gz", hash = "sha256:df81779732feb9d01e5d513fad0122efb3d53bbc75f61b2a4f29a020bc985e70"}, ] +[[package]] +name = "psutil" +version = "6.1.0" +description = "Cross-platform lib for process and system monitoring in Python." +optional = false +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,>=2.7" +files = [ + {file = "psutil-6.1.0-cp27-cp27m-macosx_10_9_x86_64.whl", hash = "sha256:ff34df86226c0227c52f38b919213157588a678d049688eded74c76c8ba4a5d0"}, + {file = "psutil-6.1.0-cp27-cp27m-manylinux2010_i686.whl", hash = "sha256:c0e0c00aa18ca2d3b2b991643b799a15fc8f0563d2ebb6040f64ce8dc027b942"}, + {file = "psutil-6.1.0-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:000d1d1ebd634b4efb383f4034437384e44a6d455260aaee2eca1e9c1b55f047"}, + {file = "psutil-6.1.0-cp27-cp27mu-manylinux2010_i686.whl", hash = "sha256:5cd2bcdc75b452ba2e10f0e8ecc0b57b827dd5d7aaffbc6821b2a9a242823a76"}, + {file = "psutil-6.1.0-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:045f00a43c737f960d273a83973b2511430d61f283a44c96bf13a6e829ba8fdc"}, + {file = "psutil-6.1.0-cp27-none-win32.whl", hash = "sha256:9118f27452b70bb1d9ab3198c1f626c2499384935aaf55388211ad982611407e"}, + {file = "psutil-6.1.0-cp27-none-win_amd64.whl", hash = "sha256:a8506f6119cff7015678e2bce904a4da21025cc70ad283a53b099e7620061d85"}, + {file = "psutil-6.1.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:6e2dcd475ce8b80522e51d923d10c7871e45f20918e027ab682f94f1c6351688"}, + {file = "psutil-6.1.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:0895b8414afafc526712c498bd9de2b063deaac4021a3b3c34566283464aff8e"}, + {file = "psutil-6.1.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9dcbfce5d89f1d1f2546a2090f4fcf87c7f669d1d90aacb7d7582addece9fb38"}, + {file = "psutil-6.1.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:498c6979f9c6637ebc3a73b3f87f9eb1ec24e1ce53a7c5173b8508981614a90b"}, + {file = "psutil-6.1.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d905186d647b16755a800e7263d43df08b790d709d575105d419f8b6ef65423a"}, + {file = "psutil-6.1.0-cp36-cp36m-win32.whl", hash = "sha256:6d3fbbc8d23fcdcb500d2c9f94e07b1342df8ed71b948a2649b5cb060a7c94ca"}, + {file = "psutil-6.1.0-cp36-cp36m-win_amd64.whl", hash = "sha256:1209036fbd0421afde505a4879dee3b2fd7b1e14fee81c0069807adcbbcca747"}, + {file = "psutil-6.1.0-cp37-abi3-win32.whl", hash = "sha256:1ad45a1f5d0b608253b11508f80940985d1d0c8f6111b5cb637533a0e6ddc13e"}, + {file = "psutil-6.1.0-cp37-abi3-win_amd64.whl", hash = "sha256:a8fb3752b491d246034fa4d279ff076501588ce8cbcdbb62c32fd7a377d996be"}, + {file = "psutil-6.1.0.tar.gz", hash = "sha256:353815f59a7f64cdaca1c0307ee13558a0512f6db064e92fe833784f08539c7a"}, +] + +[package.extras] +dev = ["black", "check-manifest", "coverage", "packaging", "pylint", "pyperf", "pypinfo", "pytest-cov", "requests", "rstcheck", "ruff", "sphinx", "sphinx_rtd_theme", "toml-sort", "twine", "virtualenv", "wheel"] +test = ["pytest", "pytest-xdist", "setuptools"] + [[package]] name = "pyarrow" version = "15.0.2" @@ -3815,99 +3860,99 @@ files = [ [[package]] name = "rapidfuzz" -version = "3.10.0" +version = "3.10.1" description = "rapid fuzzy string matching" optional = true python-versions = ">=3.9" files = [ - {file = "rapidfuzz-3.10.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:884453860de029380dded8f3c1918af2d8eb5adf8010261645c7e5c88c2b5428"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:718c9bd369288aca5fa929df6dbf66fdbe9768d90940a940c0b5cdc96ade4309"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a68e3724b7dab761c01816aaa64b0903734d999d5589daf97c14ef5cc0629a8e"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1af60988d47534246d9525f77288fdd9de652608a4842815d9018570b959acc6"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3084161fc3e963056232ef8d937449a2943852e07101f5a136c8f3cfa4119217"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6cd67d3d017296d98ff505529104299f78433e4b8af31b55003d901a62bbebe9"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b11a127ac590fc991e8a02c2d7e1ac86e8141c92f78546f18b5c904064a0552c"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:aadce42147fc09dcef1afa892485311e824c050352e1aa6e47f56b9b27af4cf0"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b54853c2371bf0e38d67da379519deb6fbe70055efb32f6607081641af3dc752"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:ce19887268e90ee81a3957eef5e46a70ecc000713796639f83828b950343f49e"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:f39a2a5ded23b9b9194ec45740dce57177b80f86c6d8eba953d3ff1a25c97766"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0ec338d5f4ad8d9339a88a08db5c23e7f7a52c2b2a10510c48a0cef1fb3f0ddc"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-win32.whl", hash = "sha256:56fd15ea8f4c948864fa5ebd9261c67cf7b89a1c517a0caef4df75446a7af18c"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-win_amd64.whl", hash = "sha256:43dfc5e733808962a822ff6d9c29f3039a3cfb3620706f5953e17cfe4496724c"}, - {file = "rapidfuzz-3.10.0-cp310-cp310-win_arm64.whl", hash = "sha256:ae7966f205b5a7fde93b44ca8fed37c1c8539328d7f179b1197de34eceaceb5f"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:bb0013795b40db5cf361e6f21ee7cda09627cf294977149b50e217d7fe9a2f03"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:69ef5b363afff7150a1fbe788007e307b9802a2eb6ad92ed51ab94e6ad2674c6"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c582c46b1bb0b19f1a5f4c1312f1b640c21d78c371a6615c34025b16ee56369b"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:288f6f6e7410cacb115fb851f3f18bf0e4231eb3f6cb5bd1cec0e7b25c4d039d"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c9e29a13d2fd9be3e7d8c26c7ef4ba60b5bc7efbc9dbdf24454c7e9ebba31768"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ea2da0459b951ee461bd4e02b8904890bd1c4263999d291c5cd01e6620177ad4"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:457827ba82261aa2ae6ac06a46d0043ab12ba7216b82d87ae1434ec0f29736d6"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:5d350864269d56f51ab81ab750c9259ae5cad3152c0680baef143dcec92206a1"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:a9b8f51e08c3f983d857c3889930af9ddecc768453822076683664772d87e374"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:7f3a6aa6e70fc27e4ff5c479f13cc9fc26a56347610f5f8b50396a0d344c5f55"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:803f255f10d63420979b1909ef976e7d30dec42025c9b067fc1d2040cc365a7e"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:2026651761bf83a0f31495cc0f70840d5c0d54388f41316e3f9cb51bd85e49a5"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-win32.whl", hash = "sha256:4df75b3ebbb8cfdb9bf8b213b168620b88fd92d0c16a8bc9f9234630b282db59"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-win_amd64.whl", hash = "sha256:f9f0bbfb6787b97c51516f3ccf97737d504db5d239ad44527673b81f598b84ab"}, - {file = "rapidfuzz-3.10.0-cp311-cp311-win_arm64.whl", hash = "sha256:10fdad800441b9c97d471a937ba7d42625f1b530db05e572f1cb7d401d95c893"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:7dc87073ba3a40dd65591a2100aa71602107443bf10770579ff9c8a3242edb94"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:a425a0a868cf8e9c6e93e1cda4b758cdfd314bb9a4fc916c5742c934e3613480"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a86d5d1d75e61df060c1e56596b6b0a4422a929dff19cc3dbfd5eee762c86b61"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:34f213d59219a9c3ca14e94a825f585811a68ac56b4118b4dc388b5b14afc108"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:96ad46f5f56f70fab2be9e5f3165a21be58d633b90bf6e67fc52a856695e4bcf"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9178277f72d144a6c7704d7ae7fa15b7b86f0f0796f0e1049c7b4ef748a662ef"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:76a35e9e19a7c883c422ffa378e9a04bc98cb3b29648c5831596401298ee51e6"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:8a6405d34c394c65e4f73a1d300c001f304f08e529d2ed6413b46ee3037956eb"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:bd393683129f446a75d8634306aed7e377627098a1286ff3af2a4f1736742820"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:b0445fa9880ead81f5a7d0efc0b9c977a947d8052c43519aceeaf56eabaf6843"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:c50bc308fa29767ed8f53a8d33b7633a9e14718ced038ed89d41b886e301da32"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e89605afebbd2d4b045bccfdc12a14b16fe8ccbae05f64b4b4c64a97dad1c891"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-win32.whl", hash = "sha256:2db9187f3acf3cd33424ecdbaad75414c298ecd1513470df7bda885dcb68cc15"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-win_amd64.whl", hash = "sha256:50e3d0c72ea15391ba9531ead7f2068a67c5b18a6a365fef3127583aaadd1725"}, - {file = "rapidfuzz-3.10.0-cp312-cp312-win_arm64.whl", hash = "sha256:9eac95b4278bd53115903d89118a2c908398ee8bdfd977ae844f1bd2b02b917c"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:fe5231e8afd069c742ac5b4f96344a0fe4aff52df8e53ef87faebf77f827822c"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:886882367dbc985f5736356105798f2ae6e794e671fc605476cbe2e73838a9bb"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b33e13e537e3afd1627d421a142a12bbbe601543558a391a6fae593356842f6e"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:094c26116d55bf9c53abd840d08422f20da78ec4c4723e5024322321caedca48"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:545fc04f2d592e4350f59deb0818886c1b444ffba3bec535b4fbb97191aaf769"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:916a6abf3632e592b937c3d04c00a6efadd8fd30539cdcd4e6e4d92be7ca5d90"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fb6ec40cef63b1922083d33bfef2f91fc0b0bc07b5b09bfee0b0f1717d558292"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:c77a7330dd15c7eb5fd3631dc646fc96327f98db8181138766bd14d3e905f0ba"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:949b5e9eeaa4ecb4c7e9c2a4689dddce60929dd1ff9c76a889cdbabe8bbf2171"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:b5363932a5aab67010ae1a6205c567d1ef256fb333bc23c27582481606be480c"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:5dd6eec15b13329abe66cc241b484002ecb0e17d694491c944a22410a6a9e5e2"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:79e7f98525b60b3c14524e0a4e1fedf7654657b6e02eb25f1be897ab097706f3"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-win32.whl", hash = "sha256:d29d1b9857c65f8cb3a29270732e1591b9bacf89de9d13fa764f79f07d8f1fd2"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-win_amd64.whl", hash = "sha256:fa9720e56663cc3649d62b4b5f3145e94b8f5611e8a8e1b46507777249d46aad"}, - {file = "rapidfuzz-3.10.0-cp313-cp313-win_arm64.whl", hash = "sha256:eda4c661e68dddd56c8fbfe1ca35e40dd2afd973f7ebb1605f4d151edc63dff8"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:cffbc50e0767396ed483900900dd58ce4351bc0d40e64bced8694bd41864cc71"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:c038b9939da3035afb6cb2f465f18163e8f070aba0482923ecff9443def67178"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ca366c2e2a54e2f663f4529b189fdeb6e14d419b1c78b754ec1744f3c01070d4"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7c4c82b1689b23b1b5e6a603164ed2be41b6f6de292a698b98ba2381e889eb9d"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:98f6ebe28831a482981ecfeedc8237047878424ad0c1add2c7f366ba44a20452"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4bd1a7676ee2a4c8e2f7f2550bece994f9f89e58afb96088964145a83af7408b"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ec9139baa3f85b65adc700eafa03ed04995ca8533dd56c924f0e458ffec044ab"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:26de93e6495078b6af4c4d93a42ca067b16cc0e95699526c82ab7d1025b4d3bf"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:f3a0bda83c18195c361b5500377d0767749f128564ca95b42c8849fd475bb327"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:63e4c175cbce8c3adc22dca5e6154588ae673f6c55374d156f3dac732c88d7de"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:4dd3d8443970eaa02ab5ae45ce584b061f2799cd9f7e875190e2617440c1f9d4"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:e5ddb2388610799fc46abe389600625058f2a73867e63e20107c5ad5ffa57c47"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-win32.whl", hash = "sha256:2e9be5d05cd960914024412b5406fb75a82f8562f45912ff86255acbfdbfb78e"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-win_amd64.whl", hash = "sha256:47aca565a39c9a6067927871973ca827023e8b65ba6c5747f4c228c8d7ddc04f"}, - {file = "rapidfuzz-3.10.0-cp39-cp39-win_arm64.whl", hash = "sha256:b0732343cdc4273b5921268026dd7266f75466eb21873cb7635a200d9d9c3fac"}, - {file = "rapidfuzz-3.10.0-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:f744b5eb1469bf92dd143d36570d2bdbbdc88fe5cb0b5405e53dd34f479cbd8a"}, - {file = "rapidfuzz-3.10.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:b67cc21a14327a0eb0f47bc3d7e59ec08031c7c55220ece672f9476e7a8068d3"}, - {file = "rapidfuzz-3.10.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2fe5783676f0afba4a522c80b15e99dbf4e393c149ab610308a8ef1f04c6bcc8"}, - {file = "rapidfuzz-3.10.0-pp310-pypy310_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d4688862f957c8629d557d084f20b2d803f8738b6c4066802a0b1cc472e088d9"}, - {file = "rapidfuzz-3.10.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:20bd153aacc244e4c907d772c703fea82754c4db14f8aa64d75ff81b7b8ab92d"}, - {file = "rapidfuzz-3.10.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:50484d563f8bfa723c74c944b0bb15b9e054db9c889348c8c307abcbee75ab92"}, - {file = "rapidfuzz-3.10.0-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:5897242d455461f2c5b82d7397b29341fd11e85bf3608a522177071044784ee8"}, - {file = "rapidfuzz-3.10.0-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:116c71a81e046ba56551d8ab68067ca7034d94b617545316d460a452c5c3c289"}, - {file = "rapidfuzz-3.10.0-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f0a547e4350d1fa32624d3eab51eff8cf329f4cae110b4ea0402486b1da8be40"}, - {file = "rapidfuzz-3.10.0-pp39-pypy39_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:399b9b79ccfcf50ca3bad7692bc098bb8eade88d7d5e15773b7f866c91156d0c"}, - {file = "rapidfuzz-3.10.0-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7947a425d1be3e744707ee58c6cb318b93a56e08f080722dcc0347e0b7a1bb9a"}, - {file = "rapidfuzz-3.10.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:94c48b4a2a4b1d22246f48e2b11cae01ec7d23f0c9123f8bb822839ad79d0a88"}, - {file = "rapidfuzz-3.10.0.tar.gz", hash = "sha256:6b62af27e65bb39276a66533655a2fa3c60a487b03935721c45b7809527979be"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:f17d9f21bf2f2f785d74f7b0d407805468b4c173fa3e52c86ec94436b338e74a"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:b31f358a70efc143909fb3d75ac6cd3c139cd41339aa8f2a3a0ead8315731f2b"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7f4f43f2204b56a61448ec2dd061e26fd344c404da99fb19f3458200c5874ba2"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9d81bf186a453a2757472133b24915768abc7c3964194406ed93e170e16c21cb"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3611c8f45379a12063d70075c75134f2a8bd2e4e9b8a7995112ddae95ca1c982"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3c3b537b97ac30da4b73930fa8a4fe2f79c6d1c10ad535c5c09726612cd6bed9"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:231ef1ec9cf7b59809ce3301006500b9d564ddb324635f4ea8f16b3e2a1780da"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:ed4f3adc1294834955b7e74edd3c6bd1aad5831c007f2d91ea839e76461a5879"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:7b6015da2e707bf632a71772a2dbf0703cff6525732c005ad24987fe86e8ec32"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:1b35a118d61d6f008e8e3fb3a77674d10806a8972c7b8be433d6598df4d60b01"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:bc308d79a7e877226f36bdf4e149e3ed398d8277c140be5c1fd892ec41739e6d"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:f017dbfecc172e2d0c37cf9e3d519179d71a7f16094b57430dffc496a098aa17"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-win32.whl", hash = "sha256:36c0e1483e21f918d0f2f26799fe5ac91c7b0c34220b73007301c4f831a9c4c7"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-win_amd64.whl", hash = "sha256:10746c1d4c8cd8881c28a87fd7ba0c9c102346dfe7ff1b0d021cdf093e9adbff"}, + {file = "rapidfuzz-3.10.1-cp310-cp310-win_arm64.whl", hash = "sha256:dfa64b89dcb906835e275187569e51aa9d546a444489e97aaf2cc84011565fbe"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:92958ae075c87fef393f835ed02d4fe8d5ee2059a0934c6c447ea3417dfbf0e8"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ba7521e072c53e33c384e78615d0718e645cab3c366ecd3cc8cb732befd94967"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:00d02cbd75d283c287471b5b3738b3e05c9096150f93f2d2dfa10b3d700f2db9"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:efa1582a397da038e2f2576c9cd49b842f56fde37d84a6b0200ffebc08d82350"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f12912acee1f506f974f58de9fdc2e62eea5667377a7e9156de53241c05fdba8"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:666d5d8b17becc3f53447bcb2b6b33ce6c2df78792495d1fa82b2924cd48701a"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:26f71582c0d62445067ee338ddad99b655a8f4e4ed517a90dcbfbb7d19310474"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8a2ef08b27167bcff230ffbfeedd4c4fa6353563d6aaa015d725dd3632fc3de7"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:365e4fc1a2b95082c890f5e98489b894e6bf8c338c6ac89bb6523c2ca6e9f086"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:1996feb7a61609fa842e6b5e0c549983222ffdedaf29644cc67e479902846dfe"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:cf654702f144beaa093103841a2ea6910d617d0bb3fccb1d1fd63c54dde2cd49"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ec108bf25de674781d0a9a935030ba090c78d49def3d60f8724f3fc1e8e75024"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-win32.whl", hash = "sha256:031f8b367e5d92f7a1e27f7322012f3c321c3110137b43cc3bf678505583ef48"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-win_amd64.whl", hash = "sha256:f98f36c6a1bb9a6c8bbec99ad87c8c0e364f34761739b5ea9adf7b48129ae8cf"}, + {file = "rapidfuzz-3.10.1-cp311-cp311-win_arm64.whl", hash = "sha256:f1da2028cb4e41be55ee797a82d6c1cf589442504244249dfeb32efc608edee7"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:1340b56340896bede246f612b6ecf685f661a56aabef3d2512481bfe23ac5835"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:2316515169b7b5a453f0ce3adbc46c42aa332cae9f2edb668e24d1fc92b2f2bb"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8e06fe6a12241ec1b72c0566c6b28cda714d61965d86569595ad24793d1ab259"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d99c1cd9443b19164ec185a7d752f4b4db19c066c136f028991a480720472e23"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a1d9aa156ed52d3446388ba4c2f335e312191d1ca9d1f5762ee983cf23e4ecf6"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:54bcf4efaaee8e015822be0c2c28214815f4f6b4f70d8362cfecbd58a71188ac"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0c955e32afdbfdf6e9ee663d24afb25210152d98c26d22d399712d29a9b976b"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:191633722203f5b7717efcb73a14f76f3b124877d0608c070b827c5226d0b972"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:195baad28057ec9609e40385991004e470af9ef87401e24ebe72c064431524ab"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:0fff4a6b87c07366662b62ae994ffbeadc472e72f725923f94b72a3db49f4671"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:4ffed25f9fdc0b287f30a98467493d1e1ce5b583f6317f70ec0263b3c97dbba6"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:d02cf8e5af89a9ac8f53c438ddff6d773f62c25c6619b29db96f4aae248177c0"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-win32.whl", hash = "sha256:f3bb81d4fe6a5d20650f8c0afcc8f6e1941f6fecdb434f11b874c42467baded0"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-win_amd64.whl", hash = "sha256:aaf83e9170cb1338922ae42d320699dccbbdca8ffed07faeb0b9257822c26e24"}, + {file = "rapidfuzz-3.10.1-cp312-cp312-win_arm64.whl", hash = "sha256:c5da802a0d085ad81b0f62828fb55557996c497b2d0b551bbdfeafd6d447892f"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:fc22d69a1c9cccd560a5c434c0371b2df0f47c309c635a01a913e03bbf183710"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:38b0dac2c8e057562b8f0d8ae5b663d2d6a28c5ab624de5b73cef9abb6129a24"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6fde3bbb14e92ce8fcb5c2edfff72e474d0080cadda1c97785bf4822f037a309"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9141fb0592e55f98fe9ac0f3ce883199b9c13e262e0bf40c5b18cdf926109d16"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:237bec5dd1bfc9b40bbd786cd27949ef0c0eb5fab5eb491904c6b5df59d39d3c"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:18123168cba156ab5794ea6de66db50f21bb3c66ae748d03316e71b27d907b95"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0b75fe506c8e02769cc47f5ab21ce3e09b6211d3edaa8f8f27331cb6988779be"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:9da82aa4b46973aaf9e03bb4c3d6977004648c8638febfc0f9d237e865761270"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:c34c022d5ad564f1a5a57a4a89793bd70d7bad428150fb8ff2760b223407cdcf"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:1e96c84d6c2a0ca94e15acb5399118fff669f4306beb98a6d8ec6f5dccab4412"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:e8e154b84a311263e1aca86818c962e1fa9eefdd643d1d5d197fcd2738f88cb9"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:335fee93188f8cd585552bb8057228ce0111bd227fa81bfd40b7df6b75def8ab"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-win32.whl", hash = "sha256:6729b856166a9e95c278410f73683957ea6100c8a9d0a8dbe434c49663689255"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-win_amd64.whl", hash = "sha256:0e06d99ad1ad97cb2ef7f51ec6b1fedd74a3a700e4949353871cf331d07b382a"}, + {file = "rapidfuzz-3.10.1-cp313-cp313-win_arm64.whl", hash = "sha256:8d1b7082104d596a3eb012e0549b2634ed15015b569f48879701e9d8db959dbb"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:779027d3307e1a2b1dc0c03c34df87a470a368a1a0840a9d2908baf2d4067956"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:440b5608ab12650d0390128d6858bc839ae77ffe5edf0b33a1551f2fa9860651"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:82cac41a411e07a6f3dc80dfbd33f6be70ea0abd72e99c59310819d09f07d945"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:958473c9f0bca250590200fd520b75be0dbdbc4a7327dc87a55b6d7dc8d68552"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9ef60dfa73749ef91cb6073be1a3e135f4846ec809cc115f3cbfc6fe283a5584"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a7fbac18f2c19fc983838a60611e67e3262e36859994c26f2ee85bb268de2355"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9a0d519ff39db887cd73f4e297922786d548f5c05d6b51f4e6754f452a7f4296"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:bebb7bc6aeb91cc57e4881b222484c26759ca865794187217c9dcea6c33adae6"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:fe07f8b9c3bb5c5ad1d2c66884253e03800f4189a60eb6acd6119ebaf3eb9894"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:bfa48a4a2d45a41457f0840c48e579db157a927f4e97acf6e20df8fc521c79de"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:2cf44d01bfe8ee605b7eaeecbc2b9ca64fc55765f17b304b40ed8995f69d7716"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1e6bbca9246d9eedaa1c84e04a7f555493ba324d52ae4d9f3d9ddd1b740dcd87"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-win32.whl", hash = "sha256:567f88180f2c1423b4fe3f3ad6e6310fc97b85bdba574801548597287fc07028"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-win_amd64.whl", hash = "sha256:6b2cd7c29d6ecdf0b780deb587198f13213ac01c430ada6913452fd0c40190fc"}, + {file = "rapidfuzz-3.10.1-cp39-cp39-win_arm64.whl", hash = "sha256:9f912d459e46607ce276128f52bea21ebc3e9a5ccf4cccfef30dd5bddcf47be8"}, + {file = "rapidfuzz-3.10.1-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:ac4452f182243cfab30ba4668ef2de101effaedc30f9faabb06a095a8c90fd16"}, + {file = "rapidfuzz-3.10.1-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:565c2bd4f7d23c32834652b27b51dd711814ab614b4e12add8476be4e20d1cf5"}, + {file = "rapidfuzz-3.10.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:187d9747149321607be4ccd6f9f366730078bed806178ec3eeb31d05545e9e8f"}, + {file = "rapidfuzz-3.10.1-pp310-pypy310_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:616290fb9a8fa87e48cb0326d26f98d4e29f17c3b762c2d586f2b35c1fd2034b"}, + {file = "rapidfuzz-3.10.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:073a5b107e17ebd264198b78614c0206fa438cce749692af5bc5f8f484883f50"}, + {file = "rapidfuzz-3.10.1-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:39c4983e2e2ccb9732f3ac7d81617088822f4a12291d416b09b8a1eadebb3e29"}, + {file = "rapidfuzz-3.10.1-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:ac7adee6bcf0c6fee495d877edad1540a7e0f5fc208da03ccb64734b43522d7a"}, + {file = "rapidfuzz-3.10.1-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:425f4ac80b22153d391ee3f94bc854668a0c6c129f05cf2eaf5ee74474ddb69e"}, + {file = "rapidfuzz-3.10.1-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:65a2fa13e8a219f9b5dcb9e74abe3ced5838a7327e629f426d333dfc8c5a6e66"}, + {file = "rapidfuzz-3.10.1-pp39-pypy39_pp73-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:75561f3df9a906aaa23787e9992b228b1ab69007932dc42070f747103e177ba8"}, + {file = "rapidfuzz-3.10.1-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:edd062490537e97ca125bc6c7f2b7331c2b73d21dc304615afe61ad1691e15d5"}, + {file = "rapidfuzz-3.10.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:cfcc8feccf63245a22dfdd16e222f1a39771a44b870beb748117a0e09cbb4a62"}, + {file = "rapidfuzz-3.10.1.tar.gz", hash = "sha256:5a15546d847a915b3f42dc79ef9b0c78b998b4e2c53b252e7166284066585979"}, ] [package.extras] @@ -4595,13 +4640,13 @@ test = ["pytest", "tornado (>=4.5)", "typeguard"] [[package]] name = "textual" -version = "0.84.0" +version = "0.85.0" description = "Modern Text User Interface framework" optional = false python-versions = "<4.0.0,>=3.8.1" files = [ - {file = "textual-0.84.0-py3-none-any.whl", hash = "sha256:1457d2cb66ba4ea46812355f31adbb4b693424a94e69d052e4affe1dc410ec96"}, - {file = "textual-0.84.0.tar.gz", hash = "sha256:fb89717960fea7a539823fa264252f7be1c84844e4b8d27360e6d4edb36846a8"}, + {file = "textual-0.85.0-py3-none-any.whl", hash = "sha256:8e75d023f06b242fb88233926dfb7801792f867643493096dd45dd216dc950f3"}, + {file = "textual-0.85.0.tar.gz", hash = "sha256:645c0fd0b4f61cd19383df78a1acd4f3b555e2c514cfa2f454e20692dffc10a0"}, ] [package.dependencies] @@ -4904,13 +4949,13 @@ bracex = ">=2.1.1" [[package]] name = "werkzeug" -version = "3.0.4" +version = "3.0.5" description = "The comprehensive WSGI web application library." optional = false python-versions = ">=3.8" files = [ - {file = "werkzeug-3.0.4-py3-none-any.whl", hash = "sha256:02c9eb92b7d6c06f31a782811505d2157837cea66aaede3e217c7c27c039476c"}, - {file = "werkzeug-3.0.4.tar.gz", hash = "sha256:34f2371506b250df4d4f84bfe7b0921e4762525762bbd936614909fe25cd7306"}, + {file = "werkzeug-3.0.5-py3-none-any.whl", hash = "sha256:6e589e0b303561b8b1a61d363ee05b1d7de6ca12f27a3a25269ae6ee93e363fd"}, + {file = "werkzeug-3.0.5.tar.gz", hash = "sha256:033bc3783777078517f32ae3ba9e86e9bc38bdbf139b1a5a3af9679a64ed1293"}, ] [package.dependencies] @@ -5144,4 +5189,4 @@ vector-db-based = ["cohere", "langchain", "openai", "tiktoken"] [metadata] lock-version = "2.0" python-versions = "^3.10" -content-hash = "dd9fc46d1b293c88a5424d1ac79244003726701dce8673cfbcdcd2424a9b3476" +content-hash = "95a6067db019e9dd56a9cccfdb89657ddb5723ed8c9b609e2e886c5c2868919e" diff --git a/airbyte-cdk/python/pyproject.toml b/airbyte-cdk/python/pyproject.toml index 06888387fee7..3c33b7ae3c37 100644 --- a/airbyte-cdk/python/pyproject.toml +++ b/airbyte-cdk/python/pyproject.toml @@ -37,6 +37,7 @@ jsonref = "~0.2" jsonschema = "~3.2.0" pandas = "2.2.2" pendulum = "<3.0.0" +psutil = "6.1.0" pydantic = "^2.7" pyrate-limiter = "~3.1.0" python-dateutil = "*" @@ -62,6 +63,9 @@ Sphinx = { version = "~4.2", optional = true } sphinx-rtd-theme = { version = "~1.0", optional = true } tiktoken = { version = "0.4.0", optional = true } nltk = { version = "3.8.1", optional = true } +# This will ensure that even when you run poetry install or pip install, the compatible version of numpy will always be chosen. +# airbyte-ci will try to install latest version when --use-local-cdk is used, resulting in the conflict. +numpy = "<2" unstructured = { version = "0.10.27", extras = ["docx", "pptx"], optional = true } "unstructured.pytesseract" = { version = ">=0.3.12", optional = true } pyjwt = "^2.8.0" diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/in_memory_files_source.py b/airbyte-cdk/python/unit_tests/sources/file_based/in_memory_files_source.py index 0a2681911211..f63d5bbd75b0 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/in_memory_files_source.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/in_memory_files_source.py @@ -110,6 +110,9 @@ def get_matching_files( globs, ) + def file_size(self, file: RemoteFile) -> int: + return 0 + def open_file(self, file: RemoteFile, mode: FileReadMode, encoding: Optional[str], logger: logging.Logger) -> IOBase: if self.file_type == "csv": return self._make_csv_file_contents(file.uri) diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py index 3f0579677f2a..2fff455b08ca 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py @@ -436,6 +436,55 @@ "required": ["name", "format"], }, }, + "delivery_method": { + "airbyte_hidden": True, + "title": "Delivery Method", + "default": "use_records_transfer", + "type": "object", + "order": 7, + "display_type": "radio", + "group": "advanced", + "oneOf": [ + { + "title": "Replicate Records", + "type": "object", + "properties": { + "delivery_type": { + "title": "Delivery Type", + "default": "use_records_transfer", + "const": "use_records_transfer", + "enum": [ + "use_records_transfer" + ], + "type": "string" + } + }, + "description": "Recommended - Extract and load structured records into your destination of choice. This is the classic method of moving data in Airbyte. It allows for blocking and hashing individual fields or files from a structured schema. Data can be flattened, typed and deduped depending on the destination.", + "required": [ + "delivery_type" + ] + }, + { + "title": "Copy Raw Files", + "type": "object", + "properties": { + "delivery_type": { + "title": "Delivery Type", + "default": "use_file_transfer", + "const": "use_file_transfer", + "enum": [ + "use_file_transfer" + ], + "type": "string" + } + }, + "description": "Copy raw files without parsing their contents. Bits are copied into the destination exactly as they appeared in the source. Recommended for use with unstructured text data, non-text and compressed files.", + "required": [ + "delivery_type" + ] + } + ] + }, }, "required": ["streams"], }, diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/stream/test_default_file_based_stream.py b/airbyte-cdk/python/unit_tests/sources/file_based/stream/test_default_file_based_stream.py index ae2b87d24b27..8d36e47143ff 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/stream/test_default_file_based_stream.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/stream/test_default_file_based_stream.py @@ -6,6 +6,7 @@ import unittest from datetime import datetime, timezone from typing import Any, Iterable, Iterator, Mapping +from unittest import mock from unittest.mock import Mock import pytest @@ -15,6 +16,7 @@ from airbyte_cdk.sources.file_based.discovery_policy import AbstractDiscoveryPolicy from airbyte_cdk.sources.file_based.exceptions import FileBasedErrorsCollector, FileBasedSourceError from airbyte_cdk.sources.file_based.file_based_stream_reader import AbstractFileBasedStreamReader +from airbyte_cdk.sources.file_based.file_types import BlobTransfer from airbyte_cdk.sources.file_based.file_types.file_type_parser import FileTypeParser from airbyte_cdk.sources.file_based.remote_file import RemoteFile from airbyte_cdk.sources.file_based.schema_validation_policies import AbstractSchemaValidationPolicy @@ -250,3 +252,48 @@ def test_yield_and_raise_collected(self) -> None: list(self.test_error_collector.yield_and_raise_collected()) assert parse_error.value.message == "Some errors occured while reading from the source." assert parse_error.value.internal_message == "Please check the logged errors for more information." + + +class DefaultFileBasedStreamFileTransferTest(unittest.TestCase): + _NOW = datetime(2022, 10, 22, tzinfo=timezone.utc) + _A_RECORD = {'bytes': 10, 'file_relative_path': 'relative/path/file.csv', 'file_url': '/absolute/path/file.csv'} + + def setUp(self) -> None: + self._stream_config = Mock() + self._stream_config.format = MockFormat() + self._stream_config.name = "a stream name" + self._catalog_schema = Mock() + self._stream_reader = Mock(spec=AbstractFileBasedStreamReader) + self._availability_strategy = Mock(spec=AbstractFileBasedAvailabilityStrategy) + self._discovery_policy = Mock(spec=AbstractDiscoveryPolicy) + self._parser = Mock(spec=FileTypeParser) + self._validation_policy = Mock(spec=AbstractSchemaValidationPolicy) + self._validation_policy.name = "validation policy name" + self._cursor = Mock(spec=AbstractFileBasedCursor) + + self._stream = DefaultFileBasedStream( + config=self._stream_config, + catalog_schema=self._catalog_schema, + stream_reader=self._stream_reader, + availability_strategy=self._availability_strategy, + discovery_policy=self._discovery_policy, + parsers={MockFormat: self._parser}, + validation_policy=self._validation_policy, + cursor=self._cursor, + errors_collector=FileBasedErrorsCollector(), + use_file_transfer=True + ) + + def test_when_read_records_from_slice_then_return_records(self) -> None: + """Verify that we have the new file method and data is empty""" + with mock.patch.object(BlobTransfer, "write_streams", return_value=[self._A_RECORD]): + messages = list(self._stream.read_records_from_slice({"files": [RemoteFile(uri="uri", last_modified=self._NOW)]})) + assert list(map(lambda message: message.record.file, messages)) == [self._A_RECORD] + assert list(map(lambda message: message.record.data, messages)) == [{}] + + def test_when_transform_record_then_return_updated_record(self) -> None: + file = RemoteFile(uri="uri", last_modified=self._NOW) + last_updated = int(self._NOW.timestamp()) * 1000 + transformed_record = self._stream.transform_record_for_file_transfer(self._A_RECORD, file) + assert transformed_record[self._stream.modified] == last_updated + assert transformed_record[self._stream.source_file_url] == file.uri diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_stream_reader.py b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_stream_reader.py index 5abbecc0434d..98b96941b49f 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_stream_reader.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_stream_reader.py @@ -73,6 +73,9 @@ def get_matching_files(self, globs: List[str]) -> Iterable[RemoteFile]: def open_file(self, file: RemoteFile) -> IOBase: pass + def file_size(self, file: RemoteFile) -> int: + return 0 + class TestSpec(AbstractFileBasedSpec): @classmethod