diff --git a/airbyte_cdk/models/__init__.py b/airbyte_cdk/models/__init__.py index 3fa24be49..2e2c3705e 100644 --- a/airbyte_cdk/models/__init__.py +++ b/airbyte_cdk/models/__init__.py @@ -19,6 +19,7 @@ AirbyteMessage, AirbyteProtocol, AirbyteRecordMessage, + AirbyteRecordMessageFileReference, AirbyteStateBlob, AirbyteStateMessage, AirbyteStateStats, diff --git a/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py b/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py index f57db7e14..bb8b5ebfb 100644 --- a/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py +++ b/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py @@ -150,6 +150,7 @@ def on_record(self, record: Record) -> Iterable[AirbyteMessage]: stream_name=record.stream_name, data_or_message=record.data, is_file_transfer_message=record.is_file_transfer_message, + file_reference=record.file_reference, ) stream = self._stream_name_to_instance[record.stream_name] diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 7a0063b5e..a499324a9 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -207,19 +207,9 @@ def _group_streams( # these legacy Python streams the way we do low-code streams to determine if they are concurrent compatible, # so we need to treat them as synchronous - file_uploader = None - if isinstance(declarative_stream, DeclarativeStream): - file_uploader = ( - self._constructor.create_component( - model_type=FileUploader, - component_definition=name_to_stream_mapping[declarative_stream.name][ - "file_uploader" - ], - config=config, - ) - if "file_uploader" in name_to_stream_mapping[declarative_stream.name] - else None - ) + supports_file_transfer = ( + "file_uploader" in name_to_stream_mapping[declarative_stream.name] + ) if ( isinstance(declarative_stream, DeclarativeStream) @@ -288,7 +278,6 @@ def _group_streams( declarative_stream.get_json_schema(), retriever, self.message_repository, - file_uploader, ), stream_slicer=declarative_stream.retriever.stream_slicer, ) @@ -319,7 +308,6 @@ def _group_streams( declarative_stream.get_json_schema(), retriever, self.message_repository, - file_uploader, ), stream_slicer=cursor, ) @@ -339,6 +327,7 @@ def _group_streams( else None, logger=self.logger, cursor=cursor, + supports_file_transfer=supports_file_transfer, ) ) elif ( @@ -350,7 +339,6 @@ def _group_streams( declarative_stream.get_json_schema(), declarative_stream.retriever, self.message_repository, - file_uploader, ), declarative_stream.retriever.stream_slicer, ) @@ -371,6 +359,7 @@ def _group_streams( cursor_field=None, logger=self.logger, cursor=final_state_cursor, + supports_file_transfer=supports_file_transfer, ) ) elif ( @@ -410,7 +399,6 @@ def _group_streams( declarative_stream.get_json_schema(), retriever, self.message_repository, - file_uploader, ), perpartition_cursor, ) @@ -425,6 +413,7 @@ def _group_streams( cursor_field=perpartition_cursor.cursor_field.cursor_field_key, logger=self.logger, cursor=perpartition_cursor, + supports_file_transfer=supports_file_transfer, ) ) else: diff --git a/airbyte_cdk/sources/declarative/declarative_component_schema.yaml b/airbyte_cdk/sources/declarative/declarative_component_schema.yaml index c28c8f4da..dd70929b9 100644 --- a/airbyte_cdk/sources/declarative/declarative_component_schema.yaml +++ b/airbyte_cdk/sources/declarative/declarative_component_schema.yaml @@ -1449,6 +1449,15 @@ definitions: anyOf: - "$ref": "#/definitions/CustomRecordExtractor" - "$ref": "#/definitions/DpathExtractor" + filename_extractor: + description: Defines the name to store the file. Stream name is automatically added to the file path. File unique ID can be used to avoid overwriting files. Random UUID will be used if the extractor is not provided. + type: string + interpolation_context: + - config + - record + examples: + - "{{ record.id }}/{{ record.file_name }}/" + - "{{ record.id }}_{{ record.file_name }}/" $parameters: type: object additional_properties: true diff --git a/airbyte_cdk/sources/declarative/extractors/record_selector.py b/airbyte_cdk/sources/declarative/extractors/record_selector.py index c37b8035b..73e854076 100644 --- a/airbyte_cdk/sources/declarative/extractors/record_selector.py +++ b/airbyte_cdk/sources/declarative/extractors/record_selector.py @@ -15,6 +15,7 @@ ) from airbyte_cdk.sources.declarative.interpolation import InterpolatedString from airbyte_cdk.sources.declarative.models import SchemaNormalization +from airbyte_cdk.sources.declarative.retrievers.file_uploader import FileUploader from airbyte_cdk.sources.declarative.transformations import RecordTransformation from airbyte_cdk.sources.types import Config, Record, StreamSlice, StreamState from airbyte_cdk.sources.utils.transform import TypeTransformer @@ -42,6 +43,7 @@ class RecordSelector(HttpSelector): record_filter: Optional[RecordFilter] = None transformations: List[RecordTransformation] = field(default_factory=lambda: []) transform_before_filtering: bool = False + file_uploader: Optional[FileUploader] = None def __post_init__(self, parameters: Mapping[str, Any]) -> None: self._parameters = parameters @@ -117,7 +119,10 @@ def filter_and_transform( transformed_filtered_data, schema=records_schema ) for data in normalized_data: - yield Record(data=data, stream_name=self.name, associated_slice=stream_slice) + record = Record(data=data, stream_name=self.name, associated_slice=stream_slice) + if self.file_uploader: + self.file_uploader.upload(record) + yield record def _normalize_by_schema( self, records: Iterable[Mapping[str, Any]], schema: Optional[Mapping[str, Any]] diff --git a/airbyte_cdk/sources/declarative/models/declarative_component_schema.py b/airbyte_cdk/sources/declarative/models/declarative_component_schema.py index c2bd928b7..8ccf92caa 100644 --- a/airbyte_cdk/sources/declarative/models/declarative_component_schema.py +++ b/airbyte_cdk/sources/declarative/models/declarative_component_schema.py @@ -1989,6 +1989,31 @@ class Config: parameters: Optional[Dict[str, Any]] = Field(None, alias="$parameters") +class FileUploader(BaseModel): + type: Literal["FileUploader"] + requester: Union[CustomRequester, HttpRequester] = Field( + ..., + description="Requester component that describes how to prepare HTTP requests to send to the source API.", + ) + download_target_extractor: Union[CustomRecordExtractor, DpathExtractor] = Field( + ..., + description="Responsible for fetching the url where the file is located. This is applied on each records and not on the HTTP response", + ) + file_extractor: Optional[Union[CustomRecordExtractor, DpathExtractor]] = Field( + None, + description="Responsible for fetching the content of the file. If not defined, the assumption is that the whole response body is the file content", + ) + filename_extractor: Optional[str] = Field( + None, + description="Defines the name to store the file. Stream name is automatically added to the file path. File unique ID can be used to avoid overwriting files. Random UUID will be used if the extractor is not provided.", + examples=[ + "{{ record.id }}/{{ record.file_name }}/", + "{{ record.id }}_{{ record.file_name }}/", + ], + ) + parameters: Optional[Dict[str, Any]] = Field(None, alias="$parameters") + + class DeclarativeStream(BaseModel): class Config: extra = Extra.allow @@ -2047,6 +2072,11 @@ class Config: description="Array of state migrations to be applied on the input state", title="State Migrations", ) + file_uploader: Optional[FileUploader] = Field( + None, + description="(experimental) Describes how to fetch a file", + title="File Uploader", + ) parameters: Optional[Dict[str, Any]] = Field(None, alias="$parameters") @@ -2278,22 +2308,6 @@ class StateDelegatingStream(BaseModel): parameters: Optional[Dict[str, Any]] = Field(None, alias="$parameters") -class FileUploader(BaseModel): - type: Literal["FileUploader"] - requester: Union[CustomRequester, HttpRequester] = Field( - ..., - description="Requester component that describes how to prepare HTTP requests to send to the source API.", - ) - download_target_extractor: Union[CustomRecordExtractor, DpathExtractor] = Field( - ..., - description="Responsible for fetching the url where the file is located. This is applied on each records and not on the HTTP response", - ) - file_extractor: Optional[Union[CustomRecordExtractor, DpathExtractor]] = Field( - None, - description="Responsible for fetching the content of the file. If not defined, the assumption is that the whole response body is the file content", - ) - - class SimpleRetriever(BaseModel): type: Literal["SimpleRetriever"] record_selector: RecordSelector = Field( @@ -2324,11 +2338,6 @@ class SimpleRetriever(BaseModel): description="PartitionRouter component that describes how to partition the stream, enabling incremental syncs and checkpointing.", title="Partition Router", ) - file_uploader: Optional[FileUploader] = Field( - None, - description="(experimental) Describes how to fetch a file", - title="File Uploader", - ) decoder: Optional[ Union[ CustomDecoder, @@ -2485,6 +2494,7 @@ class DynamicDeclarativeStream(BaseModel): DeclarativeSource1.update_forward_refs() DeclarativeSource2.update_forward_refs() SelectiveAuthenticator.update_forward_refs() +FileUploader.update_forward_refs() DeclarativeStream.update_forward_refs() SessionTokenAuthenticator.update_forward_refs() DynamicSchemaLoader.update_forward_refs() diff --git a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py index e01505fdc..434ad7f0e 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -1755,6 +1755,11 @@ def create_declarative_stream( transformations.append( self._create_component_from_model(model=transformation_model, config=config) ) + file_uploader = None + if model.file_uploader: + file_uploader = self._create_component_from_model( + model=model.file_uploader, config=config + ) retriever = self._create_component_from_model( model=model.retriever, @@ -1766,6 +1771,7 @@ def create_declarative_stream( stop_condition_on_cursor=stop_condition_on_cursor, client_side_incremental_sync=client_side_incremental_sync, transformations=transformations, + file_uploader=file_uploader, incremental_sync=model.incremental_sync, ) cursor_field = model.incremental_sync.cursor_field if model.incremental_sync else None @@ -2607,6 +2613,7 @@ def create_record_selector( transformations: List[RecordTransformation] | None = None, decoder: Decoder | None = None, client_side_incremental_sync: Dict[str, Any] | None = None, + file_uploader: Optional[FileUploader] = None, **kwargs: Any, ) -> RecordSelector: extractor = self._create_component_from_model( @@ -2644,6 +2651,7 @@ def create_record_selector( config=config, record_filter=record_filter, transformations=transformations or [], + file_uploader=file_uploader, schema_normalization=schema_normalization, parameters=model.parameters or {}, transform_before_filtering=transform_before_filtering, @@ -2701,6 +2709,7 @@ def create_simple_retriever( stop_condition_on_cursor: bool = False, client_side_incremental_sync: Optional[Dict[str, Any]] = None, transformations: List[RecordTransformation], + file_uploader: Optional[FileUploader] = None, incremental_sync: Optional[ Union[ IncrementingCountCursorModel, DatetimeBasedCursorModel, CustomIncrementalSyncModel @@ -2723,6 +2732,7 @@ def create_simple_retriever( decoder=decoder, transformations=transformations, client_side_incremental_sync=client_side_incremental_sync, + file_uploader=file_uploader, ) url_base = ( model.requester.url_base @@ -3338,7 +3348,13 @@ def create_file_uploader( name=name, **kwargs, ) - return FileUploader(requester, download_target_extractor) + return FileUploader( + requester=requester, + download_target_extractor=download_target_extractor, + config=config, + parameters=model.parameters or {}, + filename_extractor=model.filename_extractor if model.filename_extractor else None, + ) def create_moving_window_call_rate_policy( self, model: MovingWindowCallRatePolicyModel, config: Config, **kwargs: Any diff --git a/airbyte_cdk/sources/declarative/retrievers/file_uploader.py b/airbyte_cdk/sources/declarative/retrievers/file_uploader.py index 508a8c213..77b1f05bf 100644 --- a/airbyte_cdk/sources/declarative/retrievers/file_uploader.py +++ b/airbyte_cdk/sources/declarative/retrievers/file_uploader.py @@ -1,44 +1,89 @@ +# +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. +# + import json +import logging +import uuid +from dataclasses import InitVar, dataclass, field from pathlib import Path -from typing import Optional +from typing import Any, Mapping, Optional, Union +from airbyte_cdk.models import AirbyteRecordMessageFileReference from airbyte_cdk.sources.declarative.extractors.record_extractor import RecordExtractor +from airbyte_cdk.sources.declarative.interpolation.interpolated_string import ( + InterpolatedString, +) from airbyte_cdk.sources.declarative.partition_routers.substream_partition_router import ( SafeResponse, ) from airbyte_cdk.sources.declarative.requesters import Requester from airbyte_cdk.sources.declarative.types import Record, StreamSlice +from airbyte_cdk.sources.types import Config +from airbyte_cdk.sources.utils.files_directory import get_files_directory +logger = logging.getLogger("airbyte") + +@dataclass class FileUploader: - def __init__( - self, - requester: Requester, - download_target_extractor: RecordExtractor, - content_extractor: Optional[RecordExtractor] = None, - ) -> None: - self._requester = requester - self._download_target_extractor = download_target_extractor - self._content_extractor = content_extractor + requester: Requester + download_target_extractor: RecordExtractor + config: Config + parameters: InitVar[Mapping[str, Any]] + + filename_extractor: Optional[Union[InterpolatedString, str]] = None + content_extractor: Optional[RecordExtractor] = None + + def __post_init__(self, parameters: Mapping[str, Any]) -> None: + if self.filename_extractor: + self.filename_extractor = InterpolatedString.create( + self.filename_extractor, + parameters=parameters, + ) def upload(self, record: Record) -> None: - # TODO validate record shape - is the transformation applied at this point? mocked_response = SafeResponse() mocked_response.content = json.dumps(record.data).encode() - download_target = list(self._download_target_extractor.extract_records(mocked_response))[0] + download_target = list(self.download_target_extractor.extract_records(mocked_response))[0] if not isinstance(download_target, str): raise ValueError( f"download_target is expected to be a str but was {type(download_target)}: {download_target}" ) - response = self._requester.send_request( + response = self.requester.send_request( stream_slice=StreamSlice( partition={}, cursor_slice={}, extra_fields={"download_target": download_target} ), ) - if self._content_extractor: + if self.content_extractor: raise NotImplementedError("TODO") else: - with open(str(Path(__file__).parent / record.data["file_name"]), "ab") as f: + files_directory = Path(get_files_directory()) + + file_name = ( + self.filename_extractor.eval(self.config, record=record) + if self.filename_extractor + else str(uuid.uuid4()) + ) + file_name = file_name.lstrip("/") + file_relative_path = Path(record.stream_name) / Path(file_name) + + full_path = files_directory / file_relative_path + full_path.parent.mkdir(parents=True, exist_ok=True) + + with open(str(full_path), "wb") as f: f.write(response.content) + file_size_bytes = full_path.stat().st_size + + logger.info("File uploaded successfully") + logger.info(f"File url: {str(full_path)}") + logger.info(f"File size: {file_size_bytes / 1024} KB") + logger.info(f"File relative path: {str(file_relative_path)}") + + record.file_reference = AirbyteRecordMessageFileReference( + file_url=str(full_path), + file_relative_path=str(file_relative_path), + file_size_bytes=file_size_bytes, + ) diff --git a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py index 9e784233b..94ee03a56 100644 --- a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py +++ b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py @@ -3,7 +3,6 @@ from typing import Any, Iterable, Mapping, Optional from airbyte_cdk.sources.declarative.retrievers import Retriever -from airbyte_cdk.sources.declarative.retrievers.file_uploader import FileUploader from airbyte_cdk.sources.message import MessageRepository from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator @@ -19,7 +18,6 @@ def __init__( json_schema: Mapping[str, Any], retriever: Retriever, message_repository: MessageRepository, - file_uploader: Optional[FileUploader] = None, ) -> None: """ The DeclarativePartitionFactory takes a retriever_factory and not a retriever directly. The reason is that our components are not @@ -30,7 +28,6 @@ def __init__( self._json_schema = json_schema self._retriever = retriever self._message_repository = message_repository - self._file_uploader = file_uploader def create(self, stream_slice: StreamSlice) -> Partition: return DeclarativePartition( @@ -38,7 +35,6 @@ def create(self, stream_slice: StreamSlice) -> Partition: self._json_schema, self._retriever, self._message_repository, - self._file_uploader, stream_slice, ) @@ -50,14 +46,12 @@ def __init__( json_schema: Mapping[str, Any], retriever: Retriever, message_repository: MessageRepository, - file_uploader: Optional[FileUploader], stream_slice: StreamSlice, ): self._stream_name = stream_name self._json_schema = json_schema self._retriever = retriever self._message_repository = message_repository - self._file_uploader = file_uploader self._stream_slice = stream_slice self._hash = SliceHasher.hash(self._stream_name, self._stream_slice) @@ -73,8 +67,6 @@ def read(self) -> Iterable[Record]: associated_slice=self._stream_slice, ) ) - if self._file_uploader: - self._file_uploader.upload(record) yield record else: self._message_repository.emit_message(stream_data) diff --git a/airbyte_cdk/sources/file_based/file_types/file_transfer.py b/airbyte_cdk/sources/file_based/file_types/file_transfer.py index 154b6ff44..0c2855d41 100644 --- a/airbyte_cdk/sources/file_based/file_types/file_transfer.py +++ b/airbyte_cdk/sources/file_based/file_types/file_transfer.py @@ -8,18 +8,12 @@ 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 from airbyte_cdk.sources.file_based.remote_file import RemoteFile - -AIRBYTE_STAGING_DIRECTORY = os.getenv("AIRBYTE_STAGING_DIRECTORY", "/staging/files") -DEFAULT_LOCAL_DIRECTORY = "/tmp/airbyte-file-transfer" +from airbyte_cdk.sources.utils.files_directory import get_files_directory class FileTransfer: def __init__(self) -> None: - self._local_directory = ( - AIRBYTE_STAGING_DIRECTORY - if os.path.exists(AIRBYTE_STAGING_DIRECTORY) - else DEFAULT_LOCAL_DIRECTORY - ) + self._local_directory = get_files_directory() def get_file( self, diff --git a/airbyte_cdk/sources/streams/concurrent/default_stream.py b/airbyte_cdk/sources/streams/concurrent/default_stream.py index 7679a1eb6..54600d635 100644 --- a/airbyte_cdk/sources/streams/concurrent/default_stream.py +++ b/airbyte_cdk/sources/streams/concurrent/default_stream.py @@ -29,6 +29,7 @@ def __init__( logger: Logger, cursor: Cursor, namespace: Optional[str] = None, + supports_file_transfer: bool = False, ) -> None: self._stream_partition_generator = partition_generator self._name = name @@ -39,6 +40,7 @@ def __init__( self._logger = logger self._cursor = cursor self._namespace = namespace + self._supports_file_transfer = supports_file_transfer def generate_partitions(self) -> Iterable[Partition]: yield from self._stream_partition_generator.generate() @@ -68,6 +70,7 @@ def as_airbyte_stream(self) -> AirbyteStream: json_schema=dict(self._json_schema), supported_sync_modes=[SyncMode.full_refresh], is_resumable=False, + is_file_based=self._supports_file_transfer, ) if self._namespace: diff --git a/airbyte_cdk/sources/types.py b/airbyte_cdk/sources/types.py index 6ee7f652a..af40f2560 100644 --- a/airbyte_cdk/sources/types.py +++ b/airbyte_cdk/sources/types.py @@ -6,6 +6,7 @@ from typing import Any, ItemsView, Iterator, KeysView, List, Mapping, Optional, ValuesView +from airbyte_cdk.models import AirbyteRecordMessageFileReference from airbyte_cdk.utils.slice_hasher import SliceHasher # A FieldPointer designates a path to a field inside a mapping. For example, retrieving ["k1", "k1.2"] in the object {"k1" :{"k1.2": @@ -24,11 +25,13 @@ def __init__( stream_name: str, associated_slice: Optional[StreamSlice] = None, is_file_transfer_message: bool = False, + file_reference: Optional[AirbyteRecordMessageFileReference] = None, ): self._data = data self._associated_slice = associated_slice self.stream_name = stream_name self.is_file_transfer_message = is_file_transfer_message + self._file_reference = file_reference @property def data(self) -> Mapping[str, Any]: @@ -38,6 +41,14 @@ def data(self) -> Mapping[str, Any]: def associated_slice(self) -> Optional[StreamSlice]: return self._associated_slice + @property + def file_reference(self) -> AirbyteRecordMessageFileReference: + return self._file_reference + + @file_reference.setter + def file_reference(self, value: AirbyteRecordMessageFileReference): + self._file_reference = value + def __repr__(self) -> str: return repr(self._data) diff --git a/airbyte_cdk/sources/utils/files_directory.py b/airbyte_cdk/sources/utils/files_directory.py new file mode 100644 index 000000000..6b8dd6b79 --- /dev/null +++ b/airbyte_cdk/sources/utils/files_directory.py @@ -0,0 +1,15 @@ +# +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. +# +import os + +AIRBYTE_STAGING_DIRECTORY = os.getenv("AIRBYTE_STAGING_DIRECTORY", "/staging/files") +DEFAULT_LOCAL_DIRECTORY = "/tmp/airbyte-file-transfer" + + +def get_files_directory() -> str: + return ( + AIRBYTE_STAGING_DIRECTORY + if os.path.exists(AIRBYTE_STAGING_DIRECTORY) + else DEFAULT_LOCAL_DIRECTORY + ) diff --git a/airbyte_cdk/sources/utils/record_helper.py b/airbyte_cdk/sources/utils/record_helper.py index 3d2cbcecf..d41907cf1 100644 --- a/airbyte_cdk/sources/utils/record_helper.py +++ b/airbyte_cdk/sources/utils/record_helper.py @@ -9,6 +9,7 @@ AirbyteLogMessage, AirbyteMessage, AirbyteRecordMessage, + AirbyteRecordMessageFileReference, AirbyteTraceMessage, ) from airbyte_cdk.models import Type as MessageType @@ -23,6 +24,7 @@ def stream_data_to_airbyte_message( transformer: TypeTransformer = TypeTransformer(TransformConfig.NoTransform), schema: Optional[Mapping[str, Any]] = None, is_file_transfer_message: bool = False, + file_reference: Optional[AirbyteRecordMessageFileReference] = None, ) -> AirbyteMessage: if schema is None: schema = {} @@ -41,7 +43,12 @@ def stream_data_to_airbyte_message( stream=stream_name, file=data, emitted_at=now_millis, data={} ) else: - message = AirbyteRecordMessage(stream=stream_name, data=data, emitted_at=now_millis) + message = AirbyteRecordMessage( + stream=stream_name, + data=data, + emitted_at=now_millis, + file_reference=file_reference, + ) return AirbyteMessage(type=MessageType.RECORD, record=message) case AirbyteTraceMessage(): return AirbyteMessage(type=MessageType.TRACE, trace=data_or_message) diff --git a/poetry.lock b/poetry.lock index 992f7f8f8..936c060f7 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 2.0.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 2.1.1 and should not be changed by hand. [[package]] name = "aiohappyeyeballs" @@ -7,7 +7,7 @@ description = "Happy Eyeballs for asyncio" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "aiohappyeyeballs-2.4.4-py3-none-any.whl", hash = "sha256:a980909d50efcd44795c4afeca523296716d50cd756ddca6af8c65b996e27de8"}, {file = "aiohappyeyeballs-2.4.4.tar.gz", hash = "sha256:5fdd7d87889c63183afc18ce9271f9b0a7d32c2303e394468dd45d514a757745"}, @@ -20,7 +20,7 @@ description = "Async http client/server framework (asyncio)" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "aiohttp-3.11.11-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:a60804bff28662cbcf340a4d61598891f12eea3a66af48ecfdc975ceec21e3c8"}, {file = "aiohttp-3.11.11-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4b4fa1cb5f270fb3eab079536b764ad740bb749ce69a94d4ec30ceee1b5940d5"}, @@ -111,7 +111,7 @@ propcache = ">=0.2.0" yarl = ">=1.17.0,<2.0" [package.extras] -speedups = ["Brotli", "aiodns (>=3.2.0)", "brotlicffi"] +speedups = ["Brotli ; platform_python_implementation == \"CPython\"", "aiodns (>=3.2.0) ; sys_platform == \"linux\" or sys_platform == \"darwin\"", "brotlicffi ; platform_python_implementation != \"CPython\""] [[package]] name = "aiosignal" @@ -120,7 +120,7 @@ description = "aiosignal: a list of registered asynchronous callbacks" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"}, {file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"}, @@ -131,17 +131,21 @@ frozenlist = ">=1.1.0" [[package]] name = "airbyte-protocol-models-dataclasses" -version = "0.14.1" +version = "0.14.1337.dev1742858109" description = "Declares the Airbyte Protocol using Python Dataclasses. Dataclasses in Python have less performance overhead compared to Pydantic models, making them a more efficient choice for scenarios where speed and memory usage are critical" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ - {file = "airbyte_protocol_models_dataclasses-0.14.1-py3-none-any.whl", hash = "sha256:dfe10b32ee09e6ba9b4f17bd309e841b61cbd61ec8f80b1937ff104efd6209a9"}, - {file = "airbyte_protocol_models_dataclasses-0.14.1.tar.gz", hash = "sha256:f62a46556b82ea0d55de144983141639e8049d836dd4e0a9d7234c5b2e103c08"}, + {file = "airbyte_protocol_models_dataclasses-0.14.1337.dev1742858109-py3-none-any.whl", hash = "sha256:e9937574976a4bbe20339074e65f99654ab317b9fb1275806247aaa698d6793c"}, + {file = "airbyte_protocol_models_dataclasses-0.14.1337.dev1742858109.tar.gz", hash = "sha256:69fc693fe1e3545c38e0bf6b27aa3a0ead3ae00e57d75fdc94eb0366189f56dc"}, ] +[package.source] +type = "legacy" +url = "https://test.pypi.org/simple" +reference = "testpypi" + [[package]] name = "annotated-types" version = "0.7.0" @@ -149,7 +153,6 @@ description = "Reusable constraint types to use with typing.Annotated" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "annotated_types-0.7.0-py3-none-any.whl", hash = "sha256:1f02e8b43a8fbbc3f3e0d4f0f4bfc8131bcb4eebe8849b8e5c773f3a1c582a53"}, {file = "annotated_types-0.7.0.tar.gz", hash = "sha256:aff07c09a53a08bc8cfccb9c85b05f1aa9a2a6f23728d790723543408344ce89"}, @@ -162,7 +165,6 @@ description = "Unicode to ASCII transliteration" optional = false python-versions = ">=3.3" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "anyascii-0.3.2-py3-none-any.whl", hash = "sha256:3b3beef6fc43d9036d3b0529050b0c48bfad8bc960e9e562d7223cfb94fe45d4"}, {file = "anyascii-0.3.2.tar.gz", hash = "sha256:9d5d32ef844fe225b8bc7cba7f950534fae4da27a9bf3a6bea2cb0ea46ce4730"}, @@ -175,7 +177,6 @@ description = "High level compatibility layer for multiple asynchronous event lo optional = true python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "anyio-4.8.0-py3-none-any.whl", hash = "sha256:b5011f270ab5eb0abf13385f851315585cc37ef330dd88e27ec3d34d651fd47a"}, {file = "anyio-4.8.0.tar.gz", hash = "sha256:1d9fe889df5212298c0c0723fa20479d1b94883a2df44bd3897aa91083316f7a"}, @@ -189,7 +190,7 @@ typing_extensions = {version = ">=4.5", markers = "python_version < \"3.13\""} [package.extras] doc = ["Sphinx (>=7.4,<8.0)", "packaging", "sphinx-autodoc-typehints (>=1.2.0)", "sphinx_rtd_theme"] -test = ["anyio[trio]", "coverage[toml] (>=7)", "exceptiongroup (>=1.2.0)", "hypothesis (>=4.0)", "psutil (>=5.9)", "pytest (>=7.0)", "trustme", "truststore (>=0.9.1)", "uvloop (>=0.21)"] +test = ["anyio[trio]", "coverage[toml] (>=7)", "exceptiongroup (>=1.2.0)", "hypothesis (>=4.0)", "psutil (>=5.9)", "pytest (>=7.0)", "trustme", "truststore (>=0.9.1) ; python_version >= \"3.10\"", "uvloop (>=0.21) ; platform_python_implementation == \"CPython\" and platform_system != \"Windows\" and python_version < \"3.14\""] trio = ["trio (>=0.26.1)"] [[package]] @@ -199,7 +200,7 @@ description = "Timeout context manager for asyncio programs" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "extra == \"vector-db-based\" and python_version < \"3.11\"" +markers = "python_version < \"3.11\" and extra == \"vector-db-based\"" files = [ {file = "async-timeout-4.0.3.tar.gz", hash = "sha256:4640d96be84d82d02ed59ea2b7105a0f7b33abe8703703cd0ab0bf87c427522f"}, {file = "async_timeout-4.0.3-py3-none-any.whl", hash = "sha256:7405140ff1230c310e51dc27b3145b9092d659ce68ff733fb0cefe3ee42be028"}, @@ -212,7 +213,6 @@ description = "reference implementation of PEP 3156" optional = false python-versions = "*" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "asyncio-3.4.3-cp33-none-win32.whl", hash = "sha256:b62c9157d36187eca799c378e572c969f0da87cd5fc42ca372d92cdb06e7e1de"}, {file = "asyncio-3.4.3-cp33-none-win_amd64.whl", hash = "sha256:c46a87b48213d7464f22d9a497b9eef8c1928b68320a2fa94240f969f6fec08c"}, @@ -227,7 +227,6 @@ description = "PEP 224 implementation" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "attributes-doc-0.4.0.tar.gz", hash = "sha256:b1576c94a714e9fc2c65c47cf10d0c8e1a5f7c4f5ae7f69006be108d95cbfbfb"}, {file = "attributes_doc-0.4.0-py2.py3-none-any.whl", hash = "sha256:4c3007d9e58f3a6cb4b9c614c4d4ce2d92161581f28e594ddd8241cc3a113bdd"}, @@ -240,19 +239,18 @@ description = "Classes Without Boilerplate" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "attrs-24.3.0-py3-none-any.whl", hash = "sha256:ac96cd038792094f438ad1f6ff80837353805ac950cd2aa0e0625ef19850c308"}, {file = "attrs-24.3.0.tar.gz", hash = "sha256:8f5c07333d543103541ba7be0e2ce16eeee8130cb0b3f9238ab904ce1e85baff"}, ] [package.extras] -benchmark = ["cloudpickle", "hypothesis", "mypy (>=1.11.1)", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins", "pytest-xdist[psutil]"] -cov = ["cloudpickle", "coverage[toml] (>=5.3)", "hypothesis", "mypy (>=1.11.1)", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-xdist[psutil]"] -dev = ["cloudpickle", "hypothesis", "mypy (>=1.11.1)", "pre-commit-uv", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-xdist[psutil]"] +benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"] +cov = ["cloudpickle ; platform_python_implementation == \"CPython\"", "coverage[toml] (>=5.3)", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"] +dev = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pre-commit-uv", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"] docs = ["cogapp", "furo", "myst-parser", "sphinx", "sphinx-notfound-page", "sphinxcontrib-towncrier", "towncrier (<24.7)"] -tests = ["cloudpickle", "hypothesis", "mypy (>=1.11.1)", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-xdist[psutil]"] -tests-mypy = ["mypy (>=1.11.1)", "pytest-mypy-plugins"] +tests = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"] +tests-mypy = ["mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\""] [[package]] name = "avro" @@ -261,7 +259,7 @@ description = "Avro is a serialization and RPC framework." optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "avro-1.12.0-py2.py3-none-any.whl", hash = "sha256:9a255c72e1837341dd4f6ff57b2b6f68c0f0cecdef62dd04962e10fd33bec05b"}, {file = "avro-1.12.0.tar.gz", hash = "sha256:cad9c53b23ceed699c7af6bddced42e2c572fd6b408c257a7d4fc4e8cf2e2d6b"}, @@ -278,7 +276,6 @@ description = "Function decoration for backoff and retry" optional = false python-versions = ">=3.7,<4.0" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "backoff-2.2.1-py3-none-any.whl", hash = "sha256:63579f9a0628e06278f7e47b7d7d5b6ce20dc65c5e96a6f3ca99a6adca0396e8"}, {file = "backoff-2.2.1.tar.gz", hash = "sha256:03f829f5bb1923180821643f8753b0502c3b682293992485b0eef2807afa5cba"}, @@ -291,7 +288,7 @@ description = "Screen-scraping library" optional = true python-versions = ">=3.6.0" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "beautifulsoup4-4.12.3-py3-none-any.whl", hash = "sha256:b80878c9f40111313e55da8ba20bdba06d8fa3969fc68304167741bbf9e082ed"}, {file = "beautifulsoup4-4.12.3.tar.gz", hash = "sha256:74e3d1928edc070d21748185c46e3fb33490f22f52a3addee9aee0f4f7781051"}, @@ -314,7 +311,6 @@ description = "Bash style brace expander." optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "bracex-2.5.post1-py3-none-any.whl", hash = "sha256:13e5732fec27828d6af308628285ad358047cec36801598368cb28bc631dbaf6"}, {file = "bracex-2.5.post1.tar.gz", hash = "sha256:12c50952415bfa773d2d9ccb8e79651b8cdb1f31a42f6091b804f6ba2b4a66b6"}, @@ -327,7 +323,6 @@ description = "Extensible memoizing collections and decorators" optional = false python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "cachetools-5.5.1-py3-none-any.whl", hash = "sha256:b76651fdc3b24ead3c648bbdeeb940c1b04d365b38b4af66788f9ec4a81d42bb"}, {file = "cachetools-5.5.1.tar.gz", hash = "sha256:70f238fbba50383ef62e55c6aff6d9673175fe59f7c6782c7a0b9e38f4a9df95"}, @@ -340,7 +335,6 @@ description = "Composable complex class support for attrs and dataclasses." optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "cattrs-24.1.2-py3-none-any.whl", hash = "sha256:67c7495b760168d931a10233f979b28dc04daf853b30752246f4f8471c6d68d0"}, {file = "cattrs-24.1.2.tar.gz", hash = "sha256:8028cfe1ff5382df59dd36474a86e02d817b06eaf8af84555441bac915d2ef85"}, @@ -355,8 +349,8 @@ typing-extensions = {version = ">=4.1.0,<4.6.3 || >4.6.3", markers = "python_ver bson = ["pymongo (>=4.4.0)"] cbor2 = ["cbor2 (>=5.4.6)"] msgpack = ["msgpack (>=1.0.5)"] -msgspec = ["msgspec (>=0.18.5)"] -orjson = ["orjson (>=3.9.2)"] +msgspec = ["msgspec (>=0.18.5) ; implementation_name == \"cpython\""] +orjson = ["orjson (>=3.9.2) ; implementation_name == \"cpython\""] pyyaml = ["pyyaml (>=6.0)"] tomlkit = ["tomlkit (>=0.11.8)"] ujson = ["ujson (>=5.7.0)"] @@ -368,7 +362,6 @@ description = "Python package for providing Mozilla's CA Bundle." optional = false python-versions = ">=3.6" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "certifi-2024.12.14-py3-none-any.whl", hash = "sha256:1275f7a45be9464efc1173084eaa30f866fe2e47d389406136d332ed4967ec56"}, {file = "certifi-2024.12.14.tar.gz", hash = "sha256:b650d30f370c2b724812bee08008be0c4163b163ddaec3f2546c1caf65f191db"}, @@ -381,7 +374,7 @@ description = "Foreign Function Interface for Python calling C code." optional = false python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and platform_python_implementation != \"PyPy\"" +markers = "platform_python_implementation != \"PyPy\"" files = [ {file = "cffi-1.17.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:df8b1c11f177bc2313ec4b2d46baec87a5f3e71fc8b45dab2ee7cae86d9aba14"}, {file = "cffi-1.17.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8f2cdc858323644ab277e9bb925ad72ae0e67f69e804f4898c070998d50b1a67"}, @@ -462,7 +455,7 @@ description = "Universal encoding detector for Python 3" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "chardet-5.2.0-py3-none-any.whl", hash = "sha256:e1cf59446890a00105fe7b7912492ea04b6e6f06d4b742b2c788469e34c82970"}, {file = "chardet-5.2.0.tar.gz", hash = "sha256:1b3b6ff479a8c414bc3fa2c0852995695c4a026dcd6d0633b2dd092ca39c1cf7"}, @@ -475,7 +468,6 @@ description = "The Real First Universal Charset Detector. Open, modern and activ optional = false python-versions = ">=3.7" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "charset_normalizer-3.4.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:91b36a978b5ae0ee86c394f5a54d6ef44db1de0815eb43de826d41d21e4af3de"}, {file = "charset_normalizer-3.4.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7461baadb4dc00fd9e0acbe254e3d7d2112e7f92ced2adc96e54ef6501c5f176"}, @@ -578,7 +570,6 @@ description = "Composable command line interface toolkit" optional = false python-versions = ">=3.7" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2"}, {file = "click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a"}, @@ -594,7 +585,7 @@ description = "" optional = true python-versions = ">=3.7,<4.0" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "cohere-4.21-py3-none-any.whl", hash = "sha256:5eb81db62e78b3156e734421cc3e657054f9d9f1d68b9f38cf48fe3a8ae40dbc"}, {file = "cohere-4.21.tar.gz", hash = "sha256:f611438f409dfc5d5a0a153a585349f5a80b169c7102b5994d9999ecf8440866"}, @@ -619,7 +610,7 @@ files = [ {file = "colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6"}, {file = "colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44"}, ] -markers = {main = "(python_version <= \"3.11\" or python_version >= \"3.12\") and platform_system == \"Windows\"", dev = "(platform_system == \"Windows\" or sys_platform == \"win32\") and (python_version <= \"3.11\" or python_version >= \"3.12\")"} +markers = {main = "platform_system == \"Windows\"", dev = "platform_system == \"Windows\" or sys_platform == \"win32\""} [[package]] name = "contourpy" @@ -628,7 +619,7 @@ description = "Python library for calculating contours of 2D quadrilateral grids optional = true python-versions = ">=3.10" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "contourpy-1.3.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:a045f341a77b77e1c5de31e74e966537bba9f3c4099b35bf4c2e3939dd54cdab"}, {file = "contourpy-1.3.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:500360b77259914f7805af7462e41f9cb7ca92ad38e9f94d6c8641b089338124"}, @@ -703,7 +694,6 @@ description = "Code coverage measurement for Python" optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "coverage-7.6.10-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:5c912978f7fbf47ef99cec50c4401340436d200d41d714c7a4766f377c5b7b78"}, {file = "coverage-7.6.10-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a01ec4af7dfeb96ff0078ad9a48810bb0cc8abcb0115180c6013a6b26237626c"}, @@ -773,7 +763,7 @@ files = [ tomli = {version = "*", optional = true, markers = "python_full_version <= \"3.11.0a6\" and extra == \"toml\""} [package.extras] -toml = ["tomli"] +toml = ["tomli ; python_full_version <= \"3.11.0a6\""] [[package]] name = "cramjam" @@ -782,7 +772,7 @@ description = "Thin Python bindings to de/compression algorithms in Rust" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "cramjam-2.9.1-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:8e82464d1e00fbbb12958999b8471ba5e9f3d9711954505a0a7b378762332e6f"}, {file = "cramjam-2.9.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6d2df8a6511cc08ef1fccd2e0c65e2ebc9f57574ec8376052a76851af5398810"}, @@ -886,7 +876,6 @@ description = "cryptography is a package which provides cryptographic recipes an optional = false python-versions = "!=3.9.0,!=3.9.1,>=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "cryptography-44.0.2-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:efcfe97d1b3c79e486554efddeb8f6f53a4cdd4cf6086642784fa31fc384e1d7"}, {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:29ecec49f3ba3f3849362854b7253a9f59799e3763b0c9d0826259a88efa02f1"}, @@ -929,10 +918,10 @@ files = [ cffi = {version = ">=1.12", markers = "platform_python_implementation != \"PyPy\""} [package.extras] -docs = ["sphinx (>=5.3.0)", "sphinx-rtd-theme (>=3.0.0)"] +docs = ["sphinx (>=5.3.0)", "sphinx-rtd-theme (>=3.0.0) ; python_version >= \"3.8\""] docstest = ["pyenchant (>=3)", "readme-renderer (>=30.0)", "sphinxcontrib-spelling (>=7.3.1)"] -nox = ["nox (>=2024.4.15)", "nox[uv] (>=2024.3.2)"] -pep8test = ["check-sdist", "click (>=8.0.1)", "mypy (>=1.4)", "ruff (>=0.3.6)"] +nox = ["nox (>=2024.4.15)", "nox[uv] (>=2024.3.2) ; python_version >= \"3.8\""] +pep8test = ["check-sdist ; python_version >= \"3.8\"", "click (>=8.0.1)", "mypy (>=1.4)", "ruff (>=0.3.6)"] sdist = ["build (>=1.0.0)"] ssh = ["bcrypt (>=3.1.5)"] test = ["certifi (>=2024)", "cryptography-vectors (==44.0.2)", "pretend (>=0.7)", "pytest (>=7.4.0)", "pytest-benchmark (>=4.0)", "pytest-cov (>=2.10.1)", "pytest-xdist (>=3.5.0)"] @@ -945,7 +934,7 @@ description = "Composable style cycles" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "cycler-0.12.1-py3-none-any.whl", hash = "sha256:85cef7cff222d8644161529808465972e51340599459b8ac3ccbac5a854e0d30"}, {file = "cycler-0.12.1.tar.gz", hash = "sha256:88bb128f02ba341da8ef447245a9e138fae777f6a23943da4540077d3601eb1c"}, @@ -962,7 +951,7 @@ description = "Easily serialize dataclasses to and from JSON." optional = true python-versions = "<4.0,>=3.7" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"file-based\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "extra == \"vector-db-based\" or extra == \"file-based\"" files = [ {file = "dataclasses_json-0.6.7-py3-none-any.whl", hash = "sha256:0dbf33f26c8d5305befd61b39d2b3414e8a407bedc2834dea9b8d642666fb40a"}, {file = "dataclasses_json-0.6.7.tar.gz", hash = "sha256:b6b3e528266ea45b9535223bc53ca645f5208833c29229e847b3f26a1cc55fc0"}, @@ -979,7 +968,6 @@ description = "A command line utility to check for unused, missing and transitiv optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "deptry-0.23.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:1f2a6817a37d76e8f6b667381b7caf6ea3e6d6c18b5be24d36c625f387c79852"}, {file = "deptry-0.23.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:9601b64cc0aed42687fdd5c912d5f1e90d7f7333fb589b14e35bfdfebae866f3"}, @@ -1013,7 +1001,6 @@ description = "Filesystem-like pathing and searching for dictionaries" optional = false python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "dpath-2.2.0-py3-none-any.whl", hash = "sha256:b330a375ded0a0d2ed404440f6c6a715deae5313af40bbb01c8a41d891900576"}, {file = "dpath-2.2.0.tar.gz", hash = "sha256:34f7e630dc55ea3f219e555726f5da4b4b25f2200319c8e6902c394258dd6a3e"}, @@ -1026,7 +1013,6 @@ description = "Dynamic version generation" optional = false python-versions = ">=3.5" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "dunamai-1.23.0-py3-none-any.whl", hash = "sha256:a0906d876e92441793c6a423e16a4802752e723e9c9a5aabdc5535df02dbe041"}, {file = "dunamai-1.23.0.tar.gz", hash = "sha256:a163746de7ea5acb6dacdab3a6ad621ebc612ed1e528aaa8beedb8887fccd2c4"}, @@ -1042,7 +1028,7 @@ description = "Emoji for Python" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "emoji-2.14.1-py3-none-any.whl", hash = "sha256:35a8a486c1460addb1499e3bf7929d3889b2e2841a57401903699fef595e942b"}, {file = "emoji-2.14.1.tar.gz", hash = "sha256:f8c50043d79a2c1410ebfae833ae1868d5941a67a6cd4d18377e2eb0bd79346b"}, @@ -1058,7 +1044,7 @@ description = "An implementation of lxml.xmlfile for the standard library" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "et_xmlfile-2.0.0-py3-none-any.whl", hash = "sha256:7a91720bc756843502c3b7504c77b8fe44217c85c537d85037f0f536151b2caa"}, {file = "et_xmlfile-2.0.0.tar.gz", hash = "sha256:dab3f4764309081ce75662649be815c4c9081e88f0837825f90fd28317d4da54"}, @@ -1087,7 +1073,7 @@ description = "Fast read/write of AVRO files" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"file-based\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "extra == \"vector-db-based\" or extra == \"file-based\"" files = [ {file = "fastavro-1.8.2-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:0e08964b2e9a455d831f2557402a683d4c4d45206f2ab9ade7c69d3dc14e0e58"}, {file = "fastavro-1.8.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:401a70b1e5c7161420c6019e0c8afa88f7c8a373468591f5ec37639a903c2509"}, @@ -1129,7 +1115,7 @@ description = "Infer file type and MIME type of any file/buffer. No external dep optional = true python-versions = "*" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "filetype-1.2.0-py2.py3-none-any.whl", hash = "sha256:7ce71b6880181241cf7ac8697a2f1eb6a8bd9b429f7ad6d27b8db9ba5f1c2d25"}, {file = "filetype-1.2.0.tar.gz", hash = "sha256:66b56cd6474bf41d8c54660347d37afcc3f7d1970648de365c102ef77548aadb"}, @@ -1142,7 +1128,6 @@ description = "the modular source code checker: pep8 pyflakes and co" optional = false python-versions = ">=3.8.1" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "flake8-6.1.0-py2.py3-none-any.whl", hash = "sha256:ffdfce58ea94c6580c77888a86506937f9a1a227dfcd15f245d694ae20a6b6e5"}, {file = "flake8-6.1.0.tar.gz", hash = "sha256:d5b3857f07c030bdb5bf41c7f53799571d75c4491748a3adcd47de929e34cd23"}, @@ -1160,7 +1145,7 @@ description = "Tools to manipulate font files" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "fonttools-4.55.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:3b332ea7b7f5f3d99f9bc5a28a23c3824ae72711abf7c4e1d62fa21699fdebe7"}, {file = "fonttools-4.55.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:5d8f925909256e62152e7c3e192655dbca3ab8c3cdef7d7b436732727e80feb6"}, @@ -1215,18 +1200,18 @@ files = [ ] [package.extras] -all = ["brotli (>=1.0.1)", "brotlicffi (>=0.8.0)", "fs (>=2.2.0,<3)", "lxml (>=4.0)", "lz4 (>=1.7.4.2)", "matplotlib", "munkres", "pycairo", "scipy", "skia-pathops (>=0.5.0)", "sympy", "uharfbuzz (>=0.23.0)", "unicodedata2 (>=15.1.0)", "xattr", "zopfli (>=0.1.4)"] +all = ["brotli (>=1.0.1) ; platform_python_implementation == \"CPython\"", "brotlicffi (>=0.8.0) ; platform_python_implementation != \"CPython\"", "fs (>=2.2.0,<3)", "lxml (>=4.0)", "lz4 (>=1.7.4.2)", "matplotlib", "munkres ; platform_python_implementation == \"PyPy\"", "pycairo", "scipy ; platform_python_implementation != \"PyPy\"", "skia-pathops (>=0.5.0)", "sympy", "uharfbuzz (>=0.23.0)", "unicodedata2 (>=15.1.0) ; python_version <= \"3.12\"", "xattr ; sys_platform == \"darwin\"", "zopfli (>=0.1.4)"] graphite = ["lz4 (>=1.7.4.2)"] -interpolatable = ["munkres", "pycairo", "scipy"] +interpolatable = ["munkres ; platform_python_implementation == \"PyPy\"", "pycairo", "scipy ; platform_python_implementation != \"PyPy\""] lxml = ["lxml (>=4.0)"] pathops = ["skia-pathops (>=0.5.0)"] plot = ["matplotlib"] repacker = ["uharfbuzz (>=0.23.0)"] symfont = ["sympy"] -type1 = ["xattr"] +type1 = ["xattr ; sys_platform == \"darwin\""] ufo = ["fs (>=2.2.0,<3)"] -unicode = ["unicodedata2 (>=15.1.0)"] -woff = ["brotli (>=1.0.1)", "brotlicffi (>=0.8.0)", "zopfli (>=0.1.4)"] +unicode = ["unicodedata2 (>=15.1.0) ; python_version <= \"3.12\""] +woff = ["brotli (>=1.0.1) ; platform_python_implementation == \"CPython\"", "brotlicffi (>=0.8.0) ; platform_python_implementation != \"CPython\"", "zopfli (>=0.1.4)"] [[package]] name = "freezegun" @@ -1235,7 +1220,6 @@ description = "Let your Python tests travel through time" optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "freezegun-1.5.1-py3-none-any.whl", hash = "sha256:bf111d7138a8abe55ab48a71755673dbaa4ab87f4cff5634a4442dfec34c15f1"}, {file = "freezegun-1.5.1.tar.gz", hash = "sha256:b29dedfcda6d5e8e083ce71b2b542753ad48cfec44037b3fc79702e2980a89e9"}, @@ -1251,7 +1235,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {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"}, @@ -1354,7 +1338,6 @@ description = "GenSON is a powerful, user-friendly JSON Schema generator." optional = false python-versions = "*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "genson-1.3.0-py3-none-any.whl", hash = "sha256:468feccd00274cc7e4c09e84b08704270ba8d95232aa280f65b986139cec67f7"}, {file = "genson-1.3.0.tar.gz", hash = "sha256:e02db9ac2e3fd29e65b5286f7135762e2cd8a986537c075b06fc5f1517308e37"}, @@ -1367,7 +1350,7 @@ description = "Lightweight in-process concurrent programming" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"sql\") and (platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "(platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\") and (extra == \"vector-db-based\" or extra == \"sql\")" files = [ {file = "greenlet-3.1.1-cp310-cp310-macosx_11_0_universal2.whl", hash = "sha256:0bbae94a29c9e5c7e4a2b7f0aae5c17e8e90acbfd3bf6270eeba60c39fce3563"}, {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0fde093fb93f35ca72a556cf72c92ea3ebfda3d79fc35bb19fbe685853869a83"}, @@ -1455,7 +1438,6 @@ description = "A pure-Python, bring-your-own-I/O implementation of HTTP/1.1" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "h11-0.14.0-py3-none-any.whl", hash = "sha256:e3fe4ac4b851c468cc8363d500db52c2ead036020723024a109d37346efaa761"}, {file = "h11-0.14.0.tar.gz", hash = "sha256:8f19fbbe99e72420ff35c00b27a34cb9937e902a8b810e2c88300c6f0a3b699d"}, @@ -1468,7 +1450,6 @@ description = "A minimal low-level HTTP client." optional = true python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "httpcore-1.0.7-py3-none-any.whl", hash = "sha256:a3fff8f43dc260d5bd363d9f9cf1830fa3a458b332856f34282de498ed420edd"}, {file = "httpcore-1.0.7.tar.gz", hash = "sha256:8551cb62a169ec7162ac7be8d4817d561f60e08eaa485234898414bb5a8a0b4c"}, @@ -1491,7 +1472,6 @@ description = "The next generation HTTP client." optional = true python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "httpx-0.28.1-py3-none-any.whl", hash = "sha256:d909fcccc110f8c7faf814ca82a9a4d816bc5a6dbfea25d6591d6985b8ba59ad"}, {file = "httpx-0.28.1.tar.gz", hash = "sha256:75e98c5f16b0f35b567856f597f06ff2270a374470a5c2392242528e3e3e42fc"}, @@ -1504,7 +1484,7 @@ httpcore = "==1.*" idna = "*" [package.extras] -brotli = ["brotli", "brotlicffi"] +brotli = ["brotli ; platform_python_implementation == \"CPython\"", "brotlicffi ; platform_python_implementation != \"CPython\""] cli = ["click (==8.*)", "pygments (==2.*)", "rich (>=10,<14)"] http2 = ["h2 (>=3,<5)"] socks = ["socksio (==1.*)"] @@ -1517,7 +1497,6 @@ description = "Internationalized Domain Names in Applications (IDNA)" optional = false python-versions = ">=3.6" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3"}, {file = "idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9"}, @@ -1533,7 +1512,7 @@ description = "Read metadata from Python packages" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "importlib_metadata-6.11.0-py3-none-any.whl", hash = "sha256:f0afba6205ad8f8947c7d338b5342d5db2afbfd82f9cbef7879a9539cc12eb9b"}, {file = "importlib_metadata-6.11.0.tar.gz", hash = "sha256:1231cf92d825c9e03cfc4da076a16de6422c863558229ea0b22b675657463443"}, @@ -1545,7 +1524,7 @@ zipp = ">=0.5" [package.extras] docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (<7.2.5)", "sphinx (>=3.5)", "sphinx-lint"] perf = ["ipython"] -testing = ["flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)", "pytest-ruff"] +testing = ["flufl.flake8", "importlib-resources (>=1.3) ; python_version < \"3.9\"", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7) ; platform_python_implementation != \"PyPy\"", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1) ; platform_python_implementation != \"PyPy\"", "pytest-perf (>=0.9.2)", "pytest-ruff"] [[package]] name = "iniconfig" @@ -1554,7 +1533,6 @@ description = "brain-dead simple config-ini parsing" optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374"}, {file = "iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3"}, @@ -1567,7 +1545,6 @@ description = "An ISO 8601 date/time/duration parser and formatter" optional = false python-versions = "*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "isodate-0.6.1-py2.py3-none-any.whl", hash = "sha256:0751eece944162659049d35f4f549ed815792b38793f07cf73381c1c87cbed96"}, {file = "isodate-0.6.1.tar.gz", hash = "sha256:48c5881de7e8b0a0d648cb024c8062dc84e7b840ed81e864c7614fd3c127bde9"}, @@ -1583,7 +1560,6 @@ description = "A very fast and expressive template engine." optional = false python-versions = ">=3.7" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "jinja2-3.1.5-py3-none-any.whl", hash = "sha256:aba0f4dc9ed8013c424088f68a5c226f7d6097ed89b246d7749c2ec4175c6adb"}, {file = "jinja2-3.1.5.tar.gz", hash = "sha256:8fefff8dc3034e27bb80d67c671eb8a9bc424c0ef4c0826edbff304cceff43bb"}, @@ -1602,7 +1578,6 @@ description = "Lightweight pipelining with Python functions" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "joblib-1.4.2-py3-none-any.whl", hash = "sha256:06d478d5674cbc267e7496a410ee875abd68e4340feff4490bcb7afb88060ae6"}, {file = "joblib-1.4.2.tar.gz", hash = "sha256:2382c5816b2636fbd20a09e0f4e9dad4736765fdfb7dca582943b9c1366b3f0e"}, @@ -1615,7 +1590,6 @@ description = "Apply JSON-Patches (RFC 6902)" optional = true python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*, !=3.6.*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "jsonpatch-1.33-py2.py3-none-any.whl", hash = "sha256:0ae28c0cd062bbd8b8ecc26d7d164fbbea9652a1a3693f3b956c1eae5145dade"}, {file = "jsonpatch-1.33.tar.gz", hash = "sha256:9fcd4009c41e6d12348b4a0ff2563ba56a2923a7dfee731d004e212e1ee5030c"}, @@ -1631,7 +1605,6 @@ description = "Identify specific nodes in a JSON document (RFC 6901)" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "jsonpointer-3.0.0-py2.py3-none-any.whl", hash = "sha256:13e088adc14fca8b6aa8177c044e12701e6ad4b28ff10e65f2267a90109c9942"}, {file = "jsonpointer-3.0.0.tar.gz", hash = "sha256:2b2d729f2091522d61c3b31f82e11870f60b68f43fbc705cb76bf4b832af59ef"}, @@ -1644,7 +1617,6 @@ description = "An implementation of JSON Reference for Python" optional = false python-versions = "*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "jsonref-0.2-py3-none-any.whl", hash = "sha256:b1e82fa0b62e2c2796a13e5401fe51790b248f6d9bf9d7212a3e31a3501b291f"}, {file = "jsonref-0.2.tar.gz", hash = "sha256:f3c45b121cf6257eafabdc3a8008763aed1cd7da06dbabc59a9e4d2a5e4e6697"}, @@ -1657,7 +1629,6 @@ description = "An implementation of JSON Schema validation for Python" optional = false python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "jsonschema-4.17.3-py3-none-any.whl", hash = "sha256:a870ad254da1a8ca84b6a2905cac29d265f805acc57af304784962a2aa6508f6"}, {file = "jsonschema-4.17.3.tar.gz", hash = "sha256:0f864437ab8b6076ba6707453ef8f98a6a0d512a80e93f8abdb676f737ecb60d"}, @@ -1678,7 +1649,7 @@ description = "A fast implementation of the Cassowary constraint solver" optional = true python-versions = ">=3.10" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "kiwisolver-1.4.8-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:88c6f252f6816a73b1f8c904f7bbe02fd67c09a69f7cb8a0eecdbf5ce78e63db"}, {file = "kiwisolver-1.4.8-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c72941acb7b67138f35b879bbe85be0f6c6a70cab78fe3ef6db9c024d9223e5b"}, @@ -1769,7 +1740,7 @@ description = "Building applications with LLMs through composability" optional = true python-versions = "<4.0,>=3.8.1" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "langchain-0.1.16-py3-none-any.whl", hash = "sha256:bc074cc5e51fad79b9ead1572fc3161918d0f614a6c8f0460543d505ad249ac7"}, {file = "langchain-0.1.16.tar.gz", hash = "sha256:b6bce78f8c071baa898884accfff15c3d81da2f0dd86c20e2f4c80b41463f49f"}, @@ -1798,11 +1769,11 @@ cli = ["typer (>=0.9.0,<0.10.0)"] cohere = ["cohere (>=4,<6)"] docarray = ["docarray[hnswlib] (>=0.32.0,<0.33.0)"] embeddings = ["sentence-transformers (>=2,<3)"] -extended-testing = ["aiosqlite (>=0.19.0,<0.20.0)", "aleph-alpha-client (>=2.15.0,<3.0.0)", "anthropic (>=0.3.11,<0.4.0)", "arxiv (>=1.4,<2.0)", "assemblyai (>=0.17.0,<0.18.0)", "atlassian-python-api (>=3.36.0,<4.0.0)", "beautifulsoup4 (>=4,<5)", "bibtexparser (>=1.4.0,<2.0.0)", "cassio (>=0.1.0,<0.2.0)", "chardet (>=5.1.0,<6.0.0)", "cohere (>=4,<6)", "couchbase (>=4.1.9,<5.0.0)", "dashvector (>=1.0.1,<2.0.0)", "databricks-vectorsearch (>=0.21,<0.22)", "datasets (>=2.15.0,<3.0.0)", "dgml-utils (>=0.3.0,<0.4.0)", "esprima (>=4.0.1,<5.0.0)", "faiss-cpu (>=1,<2)", "feedparser (>=6.0.10,<7.0.0)", "fireworks-ai (>=0.9.0,<0.10.0)", "geopandas (>=0.13.1,<0.14.0)", "gitpython (>=3.1.32,<4.0.0)", "google-cloud-documentai (>=2.20.1,<3.0.0)", "gql (>=3.4.1,<4.0.0)", "hologres-vector (>=0.0.6,<0.0.7)", "html2text (>=2020.1.16,<2021.0.0)", "javelin-sdk (>=0.1.8,<0.2.0)", "jinja2 (>=3,<4)", "jq (>=1.4.1,<2.0.0)", "jsonschema (>1)", "langchain-openai (>=0.0.2,<0.1)", "lxml (>=4.9.3,<6.0)", "markdownify (>=0.11.6,<0.12.0)", "motor (>=3.3.1,<4.0.0)", "msal (>=1.25.0,<2.0.0)", "mwparserfromhell (>=0.6.4,<0.7.0)", "mwxml (>=0.3.3,<0.4.0)", "newspaper3k (>=0.2.8,<0.3.0)", "numexpr (>=2.8.6,<3.0.0)", "openai (<2)", "openapi-pydantic (>=0.3.2,<0.4.0)", "pandas (>=2.0.1,<3.0.0)", "pdfminer-six (>=20221105,<20221106)", "pgvector (>=0.1.6,<0.2.0)", "praw (>=7.7.1,<8.0.0)", "psychicapi (>=0.8.0,<0.9.0)", "py-trello (>=0.19.0,<0.20.0)", "pymupdf (>=1.22.3,<2.0.0)", "pypdf (>=3.4.0,<4.0.0)", "pypdfium2 (>=4.10.0,<5.0.0)", "pyspark (>=3.4.0,<4.0.0)", "rank-bm25 (>=0.2.2,<0.3.0)", "rapidfuzz (>=3.1.1,<4.0.0)", "rapidocr-onnxruntime (>=1.3.2,<2.0.0)", "rdflib (==7.0.0)", "requests-toolbelt (>=1.0.0,<2.0.0)", "rspace_client (>=2.5.0,<3.0.0)", "scikit-learn (>=1.2.2,<2.0.0)", "sqlite-vss (>=0.1.2,<0.2.0)", "streamlit (>=1.18.0,<2.0.0)", "sympy (>=1.12,<2.0)", "telethon (>=1.28.5,<2.0.0)", "timescale-vector (>=0.0.1,<0.0.2)", "tqdm (>=4.48.0)", "upstash-redis (>=0.15.0,<0.16.0)", "xata (>=1.0.0a7,<2.0.0)", "xmltodict (>=0.13.0,<0.14.0)"] +extended-testing = ["aiosqlite (>=0.19.0,<0.20.0)", "aleph-alpha-client (>=2.15.0,<3.0.0)", "anthropic (>=0.3.11,<0.4.0)", "arxiv (>=1.4,<2.0)", "assemblyai (>=0.17.0,<0.18.0)", "atlassian-python-api (>=3.36.0,<4.0.0)", "beautifulsoup4 (>=4,<5)", "bibtexparser (>=1.4.0,<2.0.0)", "cassio (>=0.1.0,<0.2.0)", "chardet (>=5.1.0,<6.0.0)", "cohere (>=4,<6)", "couchbase (>=4.1.9,<5.0.0)", "dashvector (>=1.0.1,<2.0.0)", "databricks-vectorsearch (>=0.21,<0.22)", "datasets (>=2.15.0,<3.0.0)", "dgml-utils (>=0.3.0,<0.4.0)", "esprima (>=4.0.1,<5.0.0)", "faiss-cpu (>=1,<2)", "feedparser (>=6.0.10,<7.0.0)", "fireworks-ai (>=0.9.0,<0.10.0)", "geopandas (>=0.13.1,<0.14.0)", "gitpython (>=3.1.32,<4.0.0)", "google-cloud-documentai (>=2.20.1,<3.0.0)", "gql (>=3.4.1,<4.0.0)", "hologres-vector (>=0.0.6,<0.0.7)", "html2text (>=2020.1.16,<2021.0.0)", "javelin-sdk (>=0.1.8,<0.2.0)", "jinja2 (>=3,<4)", "jq (>=1.4.1,<2.0.0)", "jsonschema (>1)", "langchain-openai (>=0.0.2,<0.1)", "lxml (>=4.9.3,<6.0)", "markdownify (>=0.11.6,<0.12.0)", "motor (>=3.3.1,<4.0.0)", "msal (>=1.25.0,<2.0.0)", "mwparserfromhell (>=0.6.4,<0.7.0)", "mwxml (>=0.3.3,<0.4.0)", "newspaper3k (>=0.2.8,<0.3.0)", "numexpr (>=2.8.6,<3.0.0)", "openai (<2)", "openapi-pydantic (>=0.3.2,<0.4.0)", "pandas (>=2.0.1,<3.0.0)", "pdfminer-six (>=20221105,<20221106)", "pgvector (>=0.1.6,<0.2.0)", "praw (>=7.7.1,<8.0.0)", "psychicapi (>=0.8.0,<0.9.0)", "py-trello (>=0.19.0,<0.20.0)", "pymupdf (>=1.22.3,<2.0.0)", "pypdf (>=3.4.0,<4.0.0)", "pypdfium2 (>=4.10.0,<5.0.0)", "pyspark (>=3.4.0,<4.0.0)", "rank-bm25 (>=0.2.2,<0.3.0)", "rapidfuzz (>=3.1.1,<4.0.0)", "rapidocr-onnxruntime (>=1.3.2,<2.0.0) ; python_full_version >= \"3.8.1\" and python_version < \"3.12\"", "rdflib (==7.0.0)", "requests-toolbelt (>=1.0.0,<2.0.0)", "rspace_client (>=2.5.0,<3.0.0)", "scikit-learn (>=1.2.2,<2.0.0)", "sqlite-vss (>=0.1.2,<0.2.0)", "streamlit (>=1.18.0,<2.0.0) ; python_full_version >= \"3.8.1\" and python_full_version != \"3.9.7\" and python_version < \"4.0\"", "sympy (>=1.12,<2.0)", "telethon (>=1.28.5,<2.0.0)", "timescale-vector (>=0.0.1,<0.0.2)", "tqdm (>=4.48.0)", "upstash-redis (>=0.15.0,<0.16.0)", "xata (>=1.0.0a7,<2.0.0)", "xmltodict (>=0.13.0,<0.14.0)"] javascript = ["esprima (>=4.0.1,<5.0.0)"] llms = ["clarifai (>=9.1.0)", "cohere (>=4,<6)", "huggingface_hub (>=0,<1)", "manifest-ml (>=0.0.1,<0.0.2)", "nlpcloud (>=1,<2)", "openai (<2)", "openlm (>=0.0.5,<0.0.6)", "torch (>=1,<3)", "transformers (>=4,<5)"] -openai = ["openai (<2)", "tiktoken (>=0.3.2,<0.6.0)"] -qdrant = ["qdrant-client (>=1.3.1,<2.0.0)"] +openai = ["openai (<2)", "tiktoken (>=0.3.2,<0.6.0) ; python_version >= \"3.9\""] +qdrant = ["qdrant-client (>=1.3.1,<2.0.0) ; python_full_version >= \"3.8.1\" and python_version < \"3.12\""] text-helpers = ["chardet (>=5.1.0,<6.0.0)"] [[package]] @@ -1812,7 +1783,7 @@ description = "Community contributed LangChain integrations." optional = true python-versions = "<4.0,>=3.8.1" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "langchain_community-0.0.32-py3-none-any.whl", hash = "sha256:406977009999952d0705de3806de2b4867e9bb8eda8ca154a59c7a8ed58da38d"}, {file = "langchain_community-0.0.32.tar.gz", hash = "sha256:1510217d646c8380f54e9850351f6d2a0b0dd73c501b666c6f4b40baa8160b29"}, @@ -1831,7 +1802,7 @@ tenacity = ">=8.1.0,<9.0.0" [package.extras] cli = ["typer (>=0.9.0,<0.10.0)"] -extended-testing = ["aiosqlite (>=0.19.0,<0.20.0)", "aleph-alpha-client (>=2.15.0,<3.0.0)", "anthropic (>=0.3.11,<0.4.0)", "arxiv (>=1.4,<2.0)", "assemblyai (>=0.17.0,<0.18.0)", "atlassian-python-api (>=3.36.0,<4.0.0)", "azure-ai-documentintelligence (>=1.0.0b1,<2.0.0)", "beautifulsoup4 (>=4,<5)", "bibtexparser (>=1.4.0,<2.0.0)", "cassio (>=0.1.0,<0.2.0)", "chardet (>=5.1.0,<6.0.0)", "cloudpickle (>=2.0.0)", "cohere (>=4,<5)", "databricks-vectorsearch (>=0.21,<0.22)", "datasets (>=2.15.0,<3.0.0)", "dgml-utils (>=0.3.0,<0.4.0)", "elasticsearch (>=8.12.0,<9.0.0)", "esprima (>=4.0.1,<5.0.0)", "faiss-cpu (>=1,<2)", "feedparser (>=6.0.10,<7.0.0)", "fireworks-ai (>=0.9.0,<0.10.0)", "friendli-client (>=1.2.4,<2.0.0)", "geopandas (>=0.13.1,<0.14.0)", "gitpython (>=3.1.32,<4.0.0)", "google-cloud-documentai (>=2.20.1,<3.0.0)", "gql (>=3.4.1,<4.0.0)", "gradientai (>=1.4.0,<2.0.0)", "hdbcli (>=2.19.21,<3.0.0)", "hologres-vector (>=0.0.6,<0.0.7)", "html2text (>=2020.1.16,<2021.0.0)", "httpx (>=0.24.1,<0.25.0)", "httpx-sse (>=0.4.0,<0.5.0)", "javelin-sdk (>=0.1.8,<0.2.0)", "jinja2 (>=3,<4)", "jq (>=1.4.1,<2.0.0)", "jsonschema (>1)", "lxml (>=4.9.3,<6.0)", "markdownify (>=0.11.6,<0.12.0)", "motor (>=3.3.1,<4.0.0)", "msal (>=1.25.0,<2.0.0)", "mwparserfromhell (>=0.6.4,<0.7.0)", "mwxml (>=0.3.3,<0.4.0)", "newspaper3k (>=0.2.8,<0.3.0)", "numexpr (>=2.8.6,<3.0.0)", "nvidia-riva-client (>=2.14.0,<3.0.0)", "oci (>=2.119.1,<3.0.0)", "openai (<2)", "openapi-pydantic (>=0.3.2,<0.4.0)", "oracle-ads (>=2.9.1,<3.0.0)", "pandas (>=2.0.1,<3.0.0)", "pdfminer-six (>=20221105,<20221106)", "pgvector (>=0.1.6,<0.2.0)", "praw (>=7.7.1,<8.0.0)", "premai (>=0.3.25,<0.4.0)", "psychicapi (>=0.8.0,<0.9.0)", "py-trello (>=0.19.0,<0.20.0)", "pyjwt (>=2.8.0,<3.0.0)", "pymupdf (>=1.22.3,<2.0.0)", "pypdf (>=3.4.0,<4.0.0)", "pypdfium2 (>=4.10.0,<5.0.0)", "pyspark (>=3.4.0,<4.0.0)", "rank-bm25 (>=0.2.2,<0.3.0)", "rapidfuzz (>=3.1.1,<4.0.0)", "rapidocr-onnxruntime (>=1.3.2,<2.0.0)", "rdflib (==7.0.0)", "requests-toolbelt (>=1.0.0,<2.0.0)", "rspace_client (>=2.5.0,<3.0.0)", "scikit-learn (>=1.2.2,<2.0.0)", "sqlite-vss (>=0.1.2,<0.2.0)", "streamlit (>=1.18.0,<2.0.0)", "sympy (>=1.12,<2.0)", "telethon (>=1.28.5,<2.0.0)", "tidb-vector (>=0.0.3,<1.0.0)", "timescale-vector (>=0.0.1,<0.0.2)", "tqdm (>=4.48.0)", "tree-sitter (>=0.20.2,<0.21.0)", "tree-sitter-languages (>=1.8.0,<2.0.0)", "upstash-redis (>=0.15.0,<0.16.0)", "vdms (>=0.0.20,<0.0.21)", "xata (>=1.0.0a7,<2.0.0)", "xmltodict (>=0.13.0,<0.14.0)"] +extended-testing = ["aiosqlite (>=0.19.0,<0.20.0)", "aleph-alpha-client (>=2.15.0,<3.0.0)", "anthropic (>=0.3.11,<0.4.0)", "arxiv (>=1.4,<2.0)", "assemblyai (>=0.17.0,<0.18.0)", "atlassian-python-api (>=3.36.0,<4.0.0)", "azure-ai-documentintelligence (>=1.0.0b1,<2.0.0)", "beautifulsoup4 (>=4,<5)", "bibtexparser (>=1.4.0,<2.0.0)", "cassio (>=0.1.0,<0.2.0)", "chardet (>=5.1.0,<6.0.0)", "cloudpickle (>=2.0.0)", "cohere (>=4,<5)", "databricks-vectorsearch (>=0.21,<0.22)", "datasets (>=2.15.0,<3.0.0)", "dgml-utils (>=0.3.0,<0.4.0)", "elasticsearch (>=8.12.0,<9.0.0)", "esprima (>=4.0.1,<5.0.0)", "faiss-cpu (>=1,<2)", "feedparser (>=6.0.10,<7.0.0)", "fireworks-ai (>=0.9.0,<0.10.0)", "friendli-client (>=1.2.4,<2.0.0)", "geopandas (>=0.13.1,<0.14.0)", "gitpython (>=3.1.32,<4.0.0)", "google-cloud-documentai (>=2.20.1,<3.0.0)", "gql (>=3.4.1,<4.0.0)", "gradientai (>=1.4.0,<2.0.0)", "hdbcli (>=2.19.21,<3.0.0)", "hologres-vector (>=0.0.6,<0.0.7)", "html2text (>=2020.1.16,<2021.0.0)", "httpx (>=0.24.1,<0.25.0)", "httpx-sse (>=0.4.0,<0.5.0)", "javelin-sdk (>=0.1.8,<0.2.0)", "jinja2 (>=3,<4)", "jq (>=1.4.1,<2.0.0)", "jsonschema (>1)", "lxml (>=4.9.3,<6.0)", "markdownify (>=0.11.6,<0.12.0)", "motor (>=3.3.1,<4.0.0)", "msal (>=1.25.0,<2.0.0)", "mwparserfromhell (>=0.6.4,<0.7.0)", "mwxml (>=0.3.3,<0.4.0)", "newspaper3k (>=0.2.8,<0.3.0)", "numexpr (>=2.8.6,<3.0.0)", "nvidia-riva-client (>=2.14.0,<3.0.0)", "oci (>=2.119.1,<3.0.0)", "openai (<2)", "openapi-pydantic (>=0.3.2,<0.4.0)", "oracle-ads (>=2.9.1,<3.0.0)", "pandas (>=2.0.1,<3.0.0)", "pdfminer-six (>=20221105,<20221106)", "pgvector (>=0.1.6,<0.2.0)", "praw (>=7.7.1,<8.0.0)", "premai (>=0.3.25,<0.4.0)", "psychicapi (>=0.8.0,<0.9.0)", "py-trello (>=0.19.0,<0.20.0)", "pyjwt (>=2.8.0,<3.0.0)", "pymupdf (>=1.22.3,<2.0.0)", "pypdf (>=3.4.0,<4.0.0)", "pypdfium2 (>=4.10.0,<5.0.0)", "pyspark (>=3.4.0,<4.0.0)", "rank-bm25 (>=0.2.2,<0.3.0)", "rapidfuzz (>=3.1.1,<4.0.0)", "rapidocr-onnxruntime (>=1.3.2,<2.0.0) ; python_full_version >= \"3.8.1\" and python_version < \"3.12\"", "rdflib (==7.0.0)", "requests-toolbelt (>=1.0.0,<2.0.0)", "rspace_client (>=2.5.0,<3.0.0)", "scikit-learn (>=1.2.2,<2.0.0)", "sqlite-vss (>=0.1.2,<0.2.0)", "streamlit (>=1.18.0,<2.0.0) ; python_full_version >= \"3.8.1\" and python_full_version != \"3.9.7\" and python_version < \"4.0\"", "sympy (>=1.12,<2.0)", "telethon (>=1.28.5,<2.0.0)", "tidb-vector (>=0.0.3,<1.0.0)", "timescale-vector (>=0.0.1,<0.0.2)", "tqdm (>=4.48.0)", "tree-sitter (>=0.20.2,<0.21.0)", "tree-sitter-languages (>=1.8.0,<2.0.0)", "upstash-redis (>=0.15.0,<0.16.0)", "vdms (>=0.0.20,<0.0.21)", "xata (>=1.0.0a7,<2.0.0)", "xmltodict (>=0.13.0,<0.14.0)"] [[package]] name = "langchain-core" @@ -1840,7 +1811,6 @@ description = "Building applications with LLMs through composability" optional = true python-versions = "<4.0,>=3.8.1" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "langchain_core-0.1.42-py3-none-any.whl", hash = "sha256:c5653ffa08a44f740295c157a24c0def4a753333f6a2c41f76bf431cd00be8b5"}, {file = "langchain_core-0.1.42.tar.gz", hash = "sha256:40751bf60ea5d8e2b2efe65290db434717ee3834870c002e40e2811f09d814e6"}, @@ -1864,7 +1834,7 @@ description = "LangChain text splitting utilities" optional = true python-versions = "<4.0,>=3.8.1" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "langchain_text_splitters-0.0.2-py3-none-any.whl", hash = "sha256:13887f32705862c1e1454213cb7834a63aae57c26fcd80346703a1d09c46168d"}, {file = "langchain_text_splitters-0.0.2.tar.gz", hash = "sha256:ac8927dc0ba08eba702f6961c9ed7df7cead8de19a9f7101ab2b5ea34201b3c1"}, @@ -1883,7 +1853,7 @@ description = "Language detection library ported from Google's language-detectio optional = true python-versions = "*" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "langdetect-1.0.9-py2-none-any.whl", hash = "sha256:7cbc0746252f19e76f77c0b1690aadf01963be835ef0cd4b56dddf2a8f1dfc2a"}, {file = "langdetect-1.0.9.tar.gz", hash = "sha256:cbc1fef89f8d062739774bd51eda3da3274006b3661d199c2655f6b3f6d605a0"}, @@ -1899,7 +1869,6 @@ description = "Client library to connect to the LangSmith LLM Tracing and Evalua optional = true python-versions = "<4.0,>=3.8.1" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "langsmith-0.1.147-py3-none-any.whl", hash = "sha256:7166fc23b965ccf839d64945a78e9f1157757add228b086141eb03a60d699a15"}, {file = "langsmith-0.1.147.tar.gz", hash = "sha256:2e933220318a4e73034657103b3b1a3a6109cc5db3566a7e8e03be8d6d7def7a"}, @@ -1925,7 +1894,6 @@ description = "Links recognition library with FULL unicode support." optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "linkify-it-py-2.0.3.tar.gz", hash = "sha256:68cda27e162e9215c17d786649d1da0021a451bdc436ef9e0fa0ba5234b9b048"}, {file = "linkify_it_py-2.0.3-py3-none-any.whl", hash = "sha256:6bcbc417b0ac14323382aef5c5192c0075bf8a9d6b41820a2b66371eac6b6d79"}, @@ -1947,7 +1915,7 @@ description = "Powerful and Pythonic XML processing library combining libxml2/li optional = true python-versions = ">=3.6" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "lxml-5.3.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:dd36439be765e2dde7660212b5275641edbc813e7b24668831a5c8ac91180656"}, {file = "lxml-5.3.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:ae5fe5c4b525aa82b8076c1a59d642c17b6e8739ecf852522c6321852178119d"}, @@ -2103,7 +2071,7 @@ description = "Python implementation of John Gruber's Markdown." optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "Markdown-3.7-py3-none-any.whl", hash = "sha256:7eb6df5690b81a1d7942992c97fad2938e956e79df20cbc6186e9c3a77b1c803"}, {file = "markdown-3.7.tar.gz", hash = "sha256:2ae2471477cfd02dbbf038d5d9bc226d40def84b4fe2986e49b59b6b472bbed2"}, @@ -2120,7 +2088,6 @@ description = "Python port of markdown-it. Markdown parsing, done right!" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "markdown-it-py-3.0.0.tar.gz", hash = "sha256:e3f60a94fa066dc52ec76661e37c851cb232d92f9886b15cb560aaada2df8feb"}, {file = "markdown_it_py-3.0.0-py3-none-any.whl", hash = "sha256:355216845c60bd96232cd8d8c40e8f9765cc86f46880e43a8fd22dc1a1a8cab1"}, @@ -2148,7 +2115,6 @@ description = "Safely add untrusted strings to HTML/XML markup." optional = false python-versions = ">=3.9" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "MarkupSafe-3.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7e94c425039cde14257288fd61dcfb01963e658efbc0ff54f5306b06054700f8"}, {file = "MarkupSafe-3.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9e2d922824181480953426608b81967de705c3cef4d1af983af849d7bd619158"}, @@ -2220,7 +2186,7 @@ description = "A lightweight library for converting complex datatypes to and fro optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"file-based\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "extra == \"vector-db-based\" or extra == \"file-based\"" files = [ {file = "marshmallow-3.25.1-py3-none-any.whl", hash = "sha256:ec5d00d873ce473b7f2ffcb7104286a376c354cab0c2fa12f5573dab03e87210"}, {file = "marshmallow-3.25.1.tar.gz", hash = "sha256:f4debda3bb11153d81ac34b0d582bf23053055ee11e791b54b4b35493468040a"}, @@ -2241,7 +2207,7 @@ description = "Python plotting package" optional = true python-versions = ">=3.10" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "matplotlib-3.10.0-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:2c5829a5a1dd5a71f0e31e6e8bb449bc0ee9dbfb05ad28fc0c6b55101b3a4be6"}, {file = "matplotlib-3.10.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a2a43cbefe22d653ab34bb55d42384ed30f611bcbdea1f8d7f431011a2e1c62e"}, @@ -2300,7 +2266,6 @@ description = "McCabe checker, plugin for flake8" optional = false python-versions = ">=3.6" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "mccabe-0.7.0-py2.py3-none-any.whl", hash = "sha256:6c2d30ab6be0e4a46919781807b4f0d834ebdd6c6e3dca0bda5a15f863427b6e"}, {file = "mccabe-0.7.0.tar.gz", hash = "sha256:348e0240c33b60bbdf4e523192ef919f28cb2c3d7d5c7794f74009290f236325"}, @@ -2313,7 +2278,6 @@ description = "Collection of plugins for markdown-it-py" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "mdit_py_plugins-0.4.2-py3-none-any.whl", hash = "sha256:0c673c3f889399a33b95e88d2f0d111b4447bdfea7f237dab2d488f459835636"}, {file = "mdit_py_plugins-0.4.2.tar.gz", hash = "sha256:5f2cd1fdb606ddf152d37ec30e46101a60512bc0e5fa1a7002c36647b09e26b5"}, @@ -2334,7 +2298,6 @@ description = "Markdown URL utilities" optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "mdurl-0.1.2-py3-none-any.whl", hash = "sha256:84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8"}, {file = "mdurl-0.1.2.tar.gz", hash = "sha256:bb413d29f5eea38f31dd4754dd7377d4465116fb207585f97bf925588687c1ba"}, @@ -2347,7 +2310,6 @@ description = "A memory profiler for Python applications" optional = false python-versions = ">=3.7.0" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "memray-1.15.0-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:9b623c0c651d611dd068236566a8a202250e3d59307c3a3f241acc47835e73eb"}, {file = "memray-1.15.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:74765f92887b7eed152e3b9f14c147c43bf0247417b18c7ea0dec173cd01633c"}, @@ -2401,10 +2363,10 @@ textual = ">=0.41.0" [package.extras] benchmark = ["asv"] -dev = ["Cython", "IPython", "asv", "black", "bump2version", "check-manifest", "flake8", "furo", "greenlet", "ipython", "isort", "mypy", "packaging", "pytest", "pytest-cov", "pytest-textual-snapshot", "setuptools", "sphinx", "sphinx-argparse", "textual (>=0.43,!=0.65.2,!=0.66)", "towncrier"] +dev = ["Cython", "IPython", "asv", "black", "bump2version", "check-manifest", "flake8", "furo", "greenlet ; python_version < \"3.14\"", "ipython", "isort", "mypy", "packaging", "pytest", "pytest-cov", "pytest-textual-snapshot", "setuptools ; python_version >= \"3.12\"", "sphinx", "sphinx-argparse", "textual (>=0.43,!=0.65.2,!=0.66)", "towncrier"] docs = ["IPython", "bump2version", "furo", "sphinx", "sphinx-argparse", "towncrier"] lint = ["black", "check-manifest", "flake8", "isort", "mypy"] -test = ["Cython", "greenlet", "ipython", "packaging", "pytest", "pytest-cov", "pytest-textual-snapshot", "setuptools", "textual (>=0.43,!=0.65.2,!=0.66)"] +test = ["Cython", "greenlet ; python_version < \"3.14\"", "ipython", "packaging", "pytest", "pytest-cov", "pytest-textual-snapshot", "setuptools ; python_version >= \"3.12\"", "textual (>=0.43,!=0.65.2,!=0.66)"] [[package]] name = "multidict" @@ -2413,7 +2375,7 @@ description = "multidict implementation" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "multidict-6.1.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:3380252550e372e8511d49481bd836264c009adb826b23fefcc5dd3c69692f60"}, {file = "multidict-6.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:99f826cbf970077383d7de805c0681799491cb939c25450b9b5b3ced03ca99f1"}, @@ -2519,7 +2481,6 @@ description = "Optional static typing for Python" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "mypy-1.14.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:52686e37cf13d559f668aa398dd7ddf1f92c5d613e4f8cb262be2fb4fedb0fcb"}, {file = "mypy-1.14.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1fb545ca340537d4b45d3eecdb3def05e913299ca72c290326be19b3804b39c0"}, @@ -2584,7 +2545,7 @@ files = [ {file = "mypy_extensions-1.0.0-py3-none-any.whl", hash = "sha256:4392f6c0eb8a5668a69e23d168ffa70f0be9ccfd32b5cc2d26a34ae5b844552d"}, {file = "mypy_extensions-1.0.0.tar.gz", hash = "sha256:75dbf8955dc00442a438fc4d0666508a9a97b6bd41aa2f0ffe9d2f2725af0782"}, ] -markers = {main = "(extra == \"vector-db-based\" or extra == \"file-based\") and (python_version <= \"3.11\" or python_version >= \"3.12\")", dev = "python_version <= \"3.11\" or python_version >= \"3.12\""} +markers = {main = "extra == \"vector-db-based\" or extra == \"file-based\""} [[package]] name = "nltk" @@ -2593,7 +2554,6 @@ description = "Natural Language Toolkit" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "nltk-3.9.1-py3-none-any.whl", hash = "sha256:4fa26829c5b00715afe3061398a8989dc643b92ce7dd93fb4585a70930d168a1"}, {file = "nltk-3.9.1.tar.gz", hash = "sha256:87d127bd3de4bd89a4f81265e5fa59cb1b199b27440175370f7417d2bc7ae868"}, @@ -2620,7 +2580,6 @@ description = "Fundamental package for array computing in Python" optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "numpy-1.26.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9ff0f4f29c51e2803569d7a51c2304de5554655a60c5d776e35b4a41413830d0"}, {file = "numpy-1.26.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2e4ee3380d6de9c9ec04745830fd9e2eccb3e6cf790d39d7b98ffd19b0dd754a"}, @@ -2667,7 +2626,7 @@ description = "Python client library for the OpenAI API" optional = true python-versions = ">=3.7.1" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "openai-0.27.9-py3-none-any.whl", hash = "sha256:6a3cf8e276d1a6262b50562fbc0cba7967cfebb78ed827d375986b48fdad6475"}, {file = "openai-0.27.9.tar.gz", hash = "sha256:b687761c82f5ebb6f61efc791b2083d2d068277b94802d4d1369efe39851813d"}, @@ -2700,7 +2659,7 @@ description = "A Python library to read/write Excel 2010 xlsx/xlsm files" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "openpyxl-3.1.5-py2.py3-none-any.whl", hash = "sha256:5282c12b107bffeef825f4617dc029afaf41d0ea60823bbb665ef3079dc79de2"}, {file = "openpyxl-3.1.5.tar.gz", hash = "sha256:cf0e3cf56142039133628b5acffe8ef0c12bc902d2aadd3e0fe5878dc08d1050"}, @@ -2716,7 +2675,6 @@ description = "Fast, correct Python JSON library supporting dataclasses, datetim optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "orjson-3.10.15-cp310-cp310-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:552c883d03ad185f720d0c09583ebde257e41b9521b74ff40e08b7dec4559c04"}, {file = "orjson-3.10.15-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:616e3e8d438d02e4854f70bfdc03a6bcdb697358dbaa6bcd19cbe24d24ece1f8"}, @@ -2806,7 +2764,6 @@ description = "Core utilities for Python packages" optional = false python-versions = ">=3.7" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "packaging-23.2-py3-none-any.whl", hash = "sha256:8c491190033a9af7e1d931d0b5dacc2ef47509b34dd0de67ed209b5203fc88c7"}, {file = "packaging-23.2.tar.gz", hash = "sha256:048fb0e9405036518eaaf48a55953c750c11e1a1b68e0dd1a9d62ed0c092cfc5"}, @@ -2819,7 +2776,6 @@ description = "Powerful data structures for data analysis, time series, and stat optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pandas-2.2.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:90c6fca2acf139569e74e8781709dccb6fe25940488755716d1d354d6bc58bce"}, {file = "pandas-2.2.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c7adfc142dac335d8c1e0dcbd37eb8617eac386596eb9e1a1b77791cf2498238"}, @@ -2894,7 +2850,7 @@ description = "Type annotations for pandas" optional = true python-versions = ">=3.10" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "pandas_stubs-2.2.3.241126-py3-none-any.whl", hash = "sha256:74aa79c167af374fe97068acc90776c0ebec5266a6e5c69fe11e9c2cf51f2267"}, {file = "pandas_stubs-2.2.3.241126.tar.gz", hash = "sha256:cf819383c6d9ae7d4dabf34cd47e1e45525bb2f312e6ad2939c2c204cb708acd"}, @@ -2911,7 +2867,6 @@ description = "Bring colors to your terminal." optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pastel-0.2.1-py2.py3-none-any.whl", hash = "sha256:4349225fcdf6c2bb34d483e523475de5bb04a5c10ef711263452cb37d7dd4364"}, {file = "pastel-0.2.1.tar.gz", hash = "sha256:e6581ac04e973cac858828c6202c1e1e81fee1dc7de7683f3e1ffe0bfd8a573d"}, @@ -2924,7 +2879,7 @@ description = "A wrapper around the pdftoppm and pdftocairo command line tools t optional = true python-versions = "*" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "pdf2image-1.16.3-py3-none-any.whl", hash = "sha256:b6154164af3677211c22cbb38b2bd778b43aca02758e962fe1e231f6d3b0e380"}, {file = "pdf2image-1.16.3.tar.gz", hash = "sha256:74208810c2cef4d9e347769b8e62a52303982ddb4f2dfd744c7ab4b940ae287e"}, @@ -2940,7 +2895,7 @@ description = "PDF parser and analyzer" optional = true python-versions = ">=3.6" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "pdfminer.six-20221105-py3-none-any.whl", hash = "sha256:1eaddd712d5b2732f8ac8486824533514f8ba12a0787b3d5fe1e686cd826532d"}, {file = "pdfminer.six-20221105.tar.gz", hash = "sha256:8448ab7b939d18b64820478ecac5394f482d7a79f5f7eaa7703c6c959c175e1d"}, @@ -2962,7 +2917,6 @@ description = "API Documentation for Python Projects" optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pdoc-15.0.1-py3-none-any.whl", hash = "sha256:fd437ab8eb55f9b942226af7865a3801e2fb731665199b74fd9a44737dbe20f9"}, {file = "pdoc-15.0.1.tar.gz", hash = "sha256:3b08382c9d312243ee6c2a1813d0ff517a6ab84d596fa2c6c6b5255b17c3d666"}, @@ -2980,7 +2934,7 @@ description = "Python Imaging Library (Fork)" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"file-based\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "extra == \"vector-db-based\" or extra == \"file-based\"" files = [ {file = "pillow-11.1.0-cp310-cp310-macosx_10_10_x86_64.whl", hash = "sha256:e1abe69aca89514737465752b4bcaf8016de61b3be1397a8fc260ba33321b3a8"}, {file = "pillow-11.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c640e5a06869c75994624551f45e5506e4256562ead981cce820d5ab39ae2192"}, @@ -3060,7 +3014,7 @@ docs = ["furo", "olefile", "sphinx (>=8.1)", "sphinx-copybutton", "sphinx-inline fpx = ["olefile"] mic = ["olefile"] tests = ["check-manifest", "coverage (>=7.4.2)", "defusedxml", "markdown2", "olefile", "packaging", "pyroma", "pytest", "pytest-cov", "pytest-timeout", "trove-classifiers (>=2024.10.12)"] -typing = ["typing-extensions"] +typing = ["typing-extensions ; python_version < \"3.10\""] xmp = ["defusedxml"] [[package]] @@ -3070,7 +3024,6 @@ description = "A small Python package for determining appropriate platform-speci optional = false python-versions = ">=3.8" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "platformdirs-4.3.6-py3-none-any.whl", hash = "sha256:73e575e1408ab8103900836b97580d5307456908a03e92031bab39e4554cc3fb"}, {file = "platformdirs-4.3.6.tar.gz", hash = "sha256:357fb2acbc885b0419afd3ce3ed34564c13c9b95c89360cd9563f73aa5e2b907"}, @@ -3088,7 +3041,7 @@ description = "An open-source, interactive data visualization library for Python optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "plotly-5.24.1-py3-none-any.whl", hash = "sha256:f67073a1e637eb0dc3e46324d9d51e2fe76e9727c892dde64ddf1e1b51f29089"}, {file = "plotly-5.24.1.tar.gz", hash = "sha256:dbc8ac8339d248a4bcc36e08a5659bacfe1b079390b8953533f4eb22169b4bae"}, @@ -3105,7 +3058,6 @@ description = "plugin and hook calling mechanisms for python" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pluggy-1.5.0-py3-none-any.whl", hash = "sha256:44e1ad92c8ca002de6377e165f3e0f1be63266ab4d554740532335b9d75ea669"}, {file = "pluggy-1.5.0.tar.gz", hash = "sha256:2cffa88e94fdc978c4c574f15f9e59b7f4201d439195c3715ca9e2486f1d0cf1"}, @@ -3122,7 +3074,6 @@ description = "A task runner that works well with poetry." optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "poethepoet-0.24.4-py3-none-any.whl", hash = "sha256:fb4ea35d7f40fe2081ea917d2e4102e2310fda2cde78974050ca83896e229075"}, {file = "poethepoet-0.24.4.tar.gz", hash = "sha256:ff4220843a87c888cbcb5312c8905214701d0af60ac7271795baa8369b428fef"}, @@ -3142,7 +3093,7 @@ description = "Accelerated property cache" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "propcache-0.2.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6b3f39a85d671436ee3d12c017f8fdea38509e4f25b28eb25877293c98c243f6"}, {file = "propcache-0.2.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:39d51fbe4285d5db5d92a929e3e21536ea3dd43732c5b177c7ef03f918dff9f2"}, @@ -3235,7 +3186,6 @@ 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" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" 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"}, @@ -3267,7 +3217,7 @@ description = "Python library for Apache Arrow" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "pyarrow-19.0.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:fc28912a2dc924dddc2087679cc8b7263accc71b9ff025a1362b004711661a69"}, {file = "pyarrow-19.0.1-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:fca15aabbe9b8355800d923cc2e82c8ef514af321e18b437c3d782aa884eaeec"}, @@ -3323,7 +3273,6 @@ description = "Python style guide checker" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pycodestyle-2.11.1-py2.py3-none-any.whl", hash = "sha256:44fe31000b2d866f2e41841b18528a505fbd7fef9017b04eff4e2648a0fadc67"}, {file = "pycodestyle-2.11.1.tar.gz", hash = "sha256:41ba0e7afc9752dfb53ced5489e89f8186be00e599e712660695b7a75ff2663f"}, @@ -3336,7 +3285,7 @@ description = "C parser in Python" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and platform_python_implementation != \"PyPy\"" +markers = "platform_python_implementation != \"PyPy\"" files = [ {file = "pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc"}, {file = "pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6"}, @@ -3349,7 +3298,6 @@ description = "Data validation using Python type hints" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pydantic-2.10.5-py3-none-any.whl", hash = "sha256:4dd4e322dbe55472cb7ca7e73f4b63574eecccf2835ffa2af9021ce113c83c53"}, {file = "pydantic-2.10.5.tar.gz", hash = "sha256:278b38dbbaec562011d659ee05f63346951b3a248a6f3642e1bc68894ea2b4ff"}, @@ -3362,7 +3310,7 @@ typing-extensions = ">=4.12.2" [package.extras] email = ["email-validator (>=2.0.0)"] -timezone = ["tzdata"] +timezone = ["tzdata ; python_version >= \"3.9\" and platform_system == \"Windows\""] [[package]] name = "pydantic-core" @@ -3371,7 +3319,6 @@ description = "Core functionality for Pydantic validation and serialization" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pydantic_core-2.27.2-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:2d367ca20b2f14095a8f4fa1210f5a7b78b8a20009ecced6b12818f455b1e9fa"}, {file = "pydantic_core-2.27.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:491a2b73db93fab69731eaee494f320faa4e093dbed776be1a829c2eb222c34c"}, @@ -3485,7 +3432,6 @@ description = "passive checker of Python programs" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pyflakes-3.1.0-py2.py3-none-any.whl", hash = "sha256:4132f6d49cb4dae6819e5379898f2b8cce3c5f23994194c24b77d5da2e36f774"}, {file = "pyflakes-3.1.0.tar.gz", hash = "sha256:a0aae034c444db0071aa077972ba4768d40c830d9539fd45bf4cd3f8f6992efc"}, @@ -3498,7 +3444,6 @@ description = "Pygments is a syntax highlighting package written in Python." optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pygments-2.19.1-py3-none-any.whl", hash = "sha256:9ea1544ad55cecf4b8242fab6dd35a93bbce657034b0611ee383099054ab6d8c"}, {file = "pygments-2.19.1.tar.gz", hash = "sha256:61c16d2a8576dc0649d9f39e089b5f02bcd27fba10d8fb4dcc28173f7a45151f"}, @@ -3514,7 +3459,6 @@ description = "JSON Web Token implementation in Python" optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "PyJWT-2.10.1-py3-none-any.whl", hash = "sha256:dcdd193e30abefd5debf142f9adfcdd2b58004e644f25406ffaebd50bd98dacb"}, {file = "pyjwt-2.10.1.tar.gz", hash = "sha256:3cc5772eb20009233caf06e9d8a0577824723b44e6648ee0a2aedb6cf9381953"}, @@ -3533,7 +3477,6 @@ description = "A development tool to measure, monitor and analyze the memory beh optional = false python-versions = ">=3.6" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "Pympler-1.1-py3-none-any.whl", hash = "sha256:5b223d6027d0619584116a0cbc28e8d2e378f7a79c1e5e024f9ff3b673c58506"}, {file = "pympler-1.1.tar.gz", hash = "sha256:1eaa867cb8992c218430f1708fdaccda53df064144d1c5656b1e6f1ee6000424"}, @@ -3549,7 +3492,7 @@ description = "pyparsing module - Classes and methods to define and execute pars optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "pyparsing-3.2.1-py3-none-any.whl", hash = "sha256:506ff4f4386c4cec0590ec19e6302d3aedb992fdc02c761e90416f158dacf8e1"}, {file = "pyparsing-3.2.1.tar.gz", hash = "sha256:61980854fd66de3a90028d679a954d5f2623e83144b5afe5ee86f43d762e5f0a"}, @@ -3565,7 +3508,6 @@ description = "pyproject-flake8 (`pflake8`), a monkey patching wrapper to connec optional = false python-versions = ">=3.8.1" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pyproject_flake8-6.1.0-py3-none-any.whl", hash = "sha256:86ea5559263c098e1aa4f866776aa2cf45362fd91a576b9fd8fbbbb55db12c4e"}, {file = "pyproject_flake8-6.1.0.tar.gz", hash = "sha256:6da8e5a264395e0148bc11844c6fb50546f1fac83ac9210f7328664135f9e70f"}, @@ -3582,7 +3524,6 @@ description = "Python Rate-Limiter using Leaky-Bucket Algorithm" optional = false python-versions = ">=3.8,<4.0" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pyrate_limiter-3.1.1-py3-none-any.whl", hash = "sha256:c51906f1d51d56dc992ff6c26e8300e32151bc6cfa3e6559792e31971dfd4e2b"}, {file = "pyrate_limiter-3.1.1.tar.gz", hash = "sha256:2f57eda712687e6eccddf6afe8f8a15b409b97ed675fe64a626058f12863b7b7"}, @@ -3599,7 +3540,6 @@ description = "Persistent/Functional/Immutable data structures" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pyrsistent-0.20.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8c3aba3e01235221e5b229a6c05f585f344734bd1ad42a8ac51493d74722bbce"}, {file = "pyrsistent-0.20.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c1beb78af5423b879edaf23c5591ff292cf7c33979734c99aa66d5914ead880f"}, @@ -3642,7 +3582,7 @@ description = "Python-tesseract is a python wrapper for Google's Tesseract-OCR" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "pytesseract-0.3.10-py3-none-any.whl", hash = "sha256:8f22cc98f765bf13517ead0c70effedb46c153540d25783e04014f28b55a5fc6"}, {file = "pytesseract-0.3.10.tar.gz", hash = "sha256:f1c3a8b0f07fd01a1085d451f5b8315be6eec1d5577a6796d46dc7a62bd4120f"}, @@ -3659,7 +3599,6 @@ description = "pytest: simple powerful testing with Python" optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pytest-7.4.4-py3-none-any.whl", hash = "sha256:b090cdf5ed60bf4c45261be03239c2c1c22df034fbffe691abe93cd80cea01d8"}, {file = "pytest-7.4.4.tar.gz", hash = "sha256:2cf0005922c6ace4a3e2ec8b4080eb0d9753fdc93107415332f50ce9e7994280"}, @@ -3683,7 +3622,6 @@ description = "Pytest plugin for measuring coverage." optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pytest-cov-6.0.0.tar.gz", hash = "sha256:fde0b595ca248bb8e2d76f020b465f3b107c9632e6a1d1705f17834c89dcadc0"}, {file = "pytest_cov-6.0.0-py3-none-any.whl", hash = "sha256:eee6f1b9e61008bd34975a4d5bab25801eb31898b032dd55addc93e96fcaaa35"}, @@ -3703,7 +3641,6 @@ description = "pytest-httpserver is a httpserver for pytest" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pytest_httpserver-1.1.1-py3-none-any.whl", hash = "sha256:aadc744bfac773a2ea93d05c2ef51fa23c087e3cc5dace3ea9d45cdd4bfe1fe8"}, {file = "pytest_httpserver-1.1.1.tar.gz", hash = "sha256:e5c46c62c0aa65e5d4331228cb2cb7db846c36e429c3e74ca806f284806bf7c6"}, @@ -3719,7 +3656,6 @@ description = "A simple plugin to use with pytest" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pytest_memray-1.7.0-py3-none-any.whl", hash = "sha256:b896718c1adf6d0cd339dfaaaa5620f035c9919e1199a79b3453804a1254306f"}, {file = "pytest_memray-1.7.0.tar.gz", hash = "sha256:c18fa907d2210b42f4096c093e2d3416dfc002dcaa450ef3f9ba819bc3dd8f5f"}, @@ -3741,7 +3677,6 @@ description = "Thin-wrapper around the mock package for easier use with pytest" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pytest-mock-3.14.0.tar.gz", hash = "sha256:2719255a1efeceadbc056d6bf3df3d1c5015530fb40cf347c0f9afac88410bd0"}, {file = "pytest_mock-3.14.0-py3-none-any.whl", hash = "sha256:0b72c38033392a5f4621342fe11e9219ac11ec9d375f8e2a0c164539e0d70f6f"}, @@ -3760,7 +3695,7 @@ description = "Python binding for Rust's library for reading excel and odf file optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "python_calamine-0.2.3-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:f292a03591b1cab1537424851b74baa33b0a55affc315248a7592ba3de1c3e83"}, {file = "python_calamine-0.2.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6cfbd23d1147f53fd70fddfb38af2a98896ecad069c9a4120e77358a6fc43b39"}, @@ -3871,7 +3806,6 @@ description = "Extensions to the standard Python datetime module" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "python-dateutil-2.9.0.post0.tar.gz", hash = "sha256:37dd54208da7e1cd875388217d5e00ebd4179249f90fb72437e91a35459a0ad3"}, {file = "python_dateutil-2.9.0.post0-py2.py3-none-any.whl", hash = "sha256:a8b2bc7bffae282281c8140a97d3aa9c14da0b136dfe83f850eea9a5f7470427"}, @@ -3887,7 +3821,7 @@ description = "Create, read, and update Microsoft Word .docx files." optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "python_docx-1.1.2-py3-none-any.whl", hash = "sha256:08c20d6058916fb19853fcf080f7f42b6270d89eac9fa5f8c15f691c0017fabe"}, {file = "python_docx-1.1.2.tar.gz", hash = "sha256:0cf1f22e95b9002addca7948e16f2cd7acdfd498047f1941ca5d293db7762efd"}, @@ -3904,7 +3838,7 @@ description = "ISO 639 language codes, names, and other associated information" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "python_iso639-2024.10.22-py3-none-any.whl", hash = "sha256:02d3ce2e01c6896b30b9cbbd3e1c8ee0d7221250b5d63ea9803e0d2a81fd1047"}, {file = "python_iso639-2024.10.22.tar.gz", hash = "sha256:750f21b6a0bc6baa24253a3d8aae92b582bf93aa40988361cd96852c2c6d9a52"}, @@ -3920,7 +3854,7 @@ description = "File type identification using libmagic" optional = true python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "python-magic-0.4.27.tar.gz", hash = "sha256:c1ba14b08e4a5f5c31a302b7721239695b2f0f058d125bd5ce1ee36b9d9d3c3b"}, {file = "python_magic-0.4.27-py2.py3-none-any.whl", hash = "sha256:c212960ad306f700aa0d01e5d7a325d20548ff97eb9920dcd29513174f0294d3"}, @@ -3933,7 +3867,7 @@ description = "Generate and manipulate Open XML PowerPoint (.pptx) files" optional = true python-versions = "*" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "python-pptx-0.6.21.tar.gz", hash = "sha256:7798a2aaf89563565b3c7120c0acfe9aff775db0db3580544e3bf4840c2e378f"}, ] @@ -3950,7 +3884,7 @@ description = "Python library for the snappy compression library from Google" optional = true python-versions = "*" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "python_snappy-0.7.3-py3-none-any.whl", hash = "sha256:074c0636cfcd97e7251330f428064050ac81a52c62ed884fc2ddebbb60ed7f50"}, {file = "python_snappy-0.7.3.tar.gz", hash = "sha256:40216c1badfb2d38ac781ecb162a1d0ec40f8ee9747e610bcfefdfa79486cee3"}, @@ -3966,7 +3900,6 @@ description = "Universally unique lexicographically sortable identifier" optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "python_ulid-3.0.0-py3-none-any.whl", hash = "sha256:e4c4942ff50dbd79167ad01ac725ec58f924b4018025ce22c858bfcff99a5e31"}, {file = "python_ulid-3.0.0.tar.gz", hash = "sha256:e50296a47dc8209d28629a22fc81ca26c00982c78934bd7766377ba37ea49a9f"}, @@ -3982,7 +3915,6 @@ description = "World timezone definitions, modern and historical" optional = false python-versions = "*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "pytz-2024.2-py2.py3-none-any.whl", hash = "sha256:31c7c1817eb7fae7ca4b8c7ee50c72f93aa2dd863de768e1ef4245d426aa0725"}, {file = "pytz-2024.2.tar.gz", hash = "sha256:2aa355083c50a0f93fa581709deac0c9ad65cca8a9e9beac660adcbd493c798a"}, @@ -3995,7 +3927,7 @@ description = "Python for Window Extensions" optional = false python-versions = "*" groups = ["dev"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and platform_system == \"Windows\"" +markers = "platform_system == \"Windows\"" files = [ {file = "pywin32-308-cp310-cp310-win32.whl", hash = "sha256:796ff4426437896550d2981b9c2ac0ffd75238ad9ea2d3bfa67a1abd546d262e"}, {file = "pywin32-308-cp310-cp310-win_amd64.whl", hash = "sha256:4fc888c59b3c0bef905ce7eb7e2106a07712015ea1c8234b703a088d46110e8e"}, @@ -4024,7 +3956,6 @@ description = "YAML parser and emitter for Python" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "PyYAML-6.0.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0a9a2848a5b7feac301353437eb7d5957887edbf81d56e903999a75a3d743086"}, {file = "PyYAML-6.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:29717114e51c84ddfba879543fb232a6ed60086602313ca38cce623c1d62cfbf"}, @@ -4088,7 +4019,6 @@ description = "rapid fuzzy string matching" optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "rapidfuzz-3.11.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:eb8a54543d16ab1b69e2c5ed96cabbff16db044a50eddfc028000138ca9ddf33"}, {file = "rapidfuzz-3.11.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:231c8b2efbd7f8d2ecd1ae900363ba168b8870644bb8f2b5aa96e4a7573bde19"}, @@ -4190,7 +4120,6 @@ description = "Alternative regular expression module, to replace re." optional = true python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "regex-2024.11.6-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:ff590880083d60acc0433f9c3f713c51f7ac6ebb9adf889c79a261ecf541aa91"}, {file = "regex-2024.11.6-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:658f90550f38270639e83ce492f27d2c8d2cd63805c65a13a14d36ca126753f0"}, @@ -4295,7 +4224,6 @@ description = "Python HTTP for Humans." optional = false python-versions = ">=3.8" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "requests-2.32.3-py3-none-any.whl", hash = "sha256:70761cfe03c773ceb22aa2f671b4757976145175cdfca038c02654d061d6dcc6"}, {file = "requests-2.32.3.tar.gz", hash = "sha256:55365417734eb18255590a9ff9eb97e9e1da868d4ccd6402399eaf68af20a760"}, @@ -4318,7 +4246,6 @@ description = "A persistent cache for python requests" optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "requests_cache-1.2.1-py3-none-any.whl", hash = "sha256:1285151cddf5331067baa82598afe2d47c7495a1334bfe7a7d329b43e9fd3603"}, {file = "requests_cache-1.2.1.tar.gz", hash = "sha256:68abc986fdc5b8d0911318fbb5f7c80eebcd4d01bfacc6685ecf8876052511d1"}, @@ -4350,7 +4277,6 @@ description = "Mock out responses from the requests package" optional = false python-versions = ">=3.5" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "requests-mock-1.12.1.tar.gz", hash = "sha256:e9e12e333b525156e82a3c852f22016b9158220d2f47454de9cae8a77d371401"}, {file = "requests_mock-1.12.1-py2.py3-none-any.whl", hash = "sha256:b1e37054004cdd5e56c84454cc7df12b25f90f382159087f4b6915aaeef39563"}, @@ -4369,7 +4295,6 @@ description = "A utility belt for advanced users of python-requests" optional = true python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "requests-toolbelt-1.0.0.tar.gz", hash = "sha256:7681a0a3d047012b5bdc0ee37d7f8f07ebe76ab08caeccfc3921ce23c88d5bc6"}, {file = "requests_toolbelt-1.0.0-py2.py3-none-any.whl", hash = "sha256:cccfdd665f0a24fcf4726e690f65639d272bb0637b9b92dfd91a5568ccf6bd06"}, @@ -4385,7 +4310,6 @@ description = "This is a small Python module for parsing Pip requirement files." optional = false python-versions = "<4.0,>=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "requirements_parser-0.11.0-py3-none-any.whl", hash = "sha256:50379eb50311834386c2568263ae5225d7b9d0867fb55cf4ecc93959de2c2684"}, {file = "requirements_parser-0.11.0.tar.gz", hash = "sha256:35f36dc969d14830bf459803da84f314dc3d17c802592e9e970f63d0359e5920"}, @@ -4402,7 +4326,6 @@ description = "Render rich text, tables, progress bars, syntax highlighting, mar optional = false python-versions = ">=3.8.0" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "rich-13.9.4-py3-none-any.whl", hash = "sha256:6049d5e6ec054bf2779ab3358186963bac2ea89175919d699e378b99738c2a90"}, {file = "rich-13.9.4.tar.gz", hash = "sha256:439594978a49a09530cff7ebc4b5c7103ef57baf48d5ea3184f21d9a2befa098"}, @@ -4423,7 +4346,6 @@ description = "An extremely fast Python linter and code formatter, written in Ru optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "ruff-0.7.4-py3-none-linux_armv6l.whl", hash = "sha256:a4919925e7684a3f18e18243cd6bea7cfb8e968a6eaa8437971f681b7ec51478"}, {file = "ruff-0.7.4-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:cfb365c135b830778dda8c04fb7d4280ed0b984e1aec27f574445231e20d6c63"}, @@ -4452,7 +4374,7 @@ description = "A set of python modules for machine learning and data mining" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "scikit_learn-1.6.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d056391530ccd1e501056160e3c9673b4da4805eb67eb2bdf4e983e1f9c9204e"}, {file = "scikit_learn-1.6.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:0c8d036eb937dbb568c6242fa598d551d88fb4399c0344d95c001980ec1c7d36"}, @@ -4508,7 +4430,7 @@ description = "Fundamental algorithms for scientific computing in Python" optional = true python-versions = ">=3.10" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "scipy-1.15.1-cp310-cp310-macosx_10_13_x86_64.whl", hash = "sha256:c64ded12dcab08afff9e805a67ff4480f5e69993310e093434b10e85dc9d43e1"}, {file = "scipy-1.15.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:5b190b935e7db569960b48840e5bef71dc513314cc4e79a1b7d14664f57fd4ff"}, @@ -4558,7 +4480,7 @@ numpy = ">=1.23.5,<2.5" [package.extras] dev = ["cython-lint (>=0.12.2)", "doit (>=0.36.0)", "mypy (==1.10.0)", "pycodestyle", "pydevtool", "rich-click", "ruff (>=0.0.292)", "types-psutil", "typing_extensions"] doc = ["intersphinx_registry", "jupyterlite-pyodide-kernel", "jupyterlite-sphinx (>=0.16.5)", "jupytext", "matplotlib (>=3.5)", "myst-nb", "numpydoc", "pooch", "pydata-sphinx-theme (>=0.15.2)", "sphinx (>=5.0.0,<8.0.0)", "sphinx-copybutton", "sphinx-design (>=0.4.0)"] -test = ["Cython", "array-api-strict (>=2.0,<2.1.1)", "asv", "gmpy2", "hypothesis (>=6.30)", "meson", "mpmath", "ninja", "pooch", "pytest", "pytest-cov", "pytest-timeout", "pytest-xdist", "scikit-umfpack", "threadpoolctl"] +test = ["Cython", "array-api-strict (>=2.0,<2.1.1)", "asv", "gmpy2", "hypothesis (>=6.30)", "meson", "mpmath", "ninja ; sys_platform != \"emscripten\"", "pooch", "pytest", "pytest-cov", "pytest-timeout", "pytest-xdist", "scikit-umfpack", "threadpoolctl"] [[package]] name = "serpyco-rs" @@ -4567,7 +4489,6 @@ description = "" optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "serpyco_rs-1.13.0-cp310-cp310-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:e722b3053e627d8a304e462bce20cae1670a2c4b0ef875b84d0de0081bec4029"}, {file = "serpyco_rs-1.13.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7f10e89c752ff78d720a42e026b0a9ada70717ad6306a9356f794280167d62bf"}, @@ -4623,20 +4544,19 @@ description = "Easily download, build, install, upgrade, and uninstall Python pa optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "setuptools-76.0.0-py3-none-any.whl", hash = "sha256:199466a166ff664970d0ee145839f5582cb9bca7a0a3a2e795b6a9cb2308e9c6"}, {file = "setuptools-76.0.0.tar.gz", hash = "sha256:43b4ee60e10b0d0ee98ad11918e114c70701bc6051662a9a675a0496c1a158f4"}, ] [package.extras] -check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)", "ruff (>=0.8.0)"] -core = ["importlib_metadata (>=6)", "jaraco.collections", "jaraco.functools (>=4)", "jaraco.text (>=3.7)", "more_itertools", "more_itertools (>=8.8)", "packaging", "packaging (>=24.2)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1)", "wheel (>=0.43.0)"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""] +core = ["importlib_metadata (>=6) ; python_version < \"3.10\"", "jaraco.collections", "jaraco.functools (>=4)", "jaraco.text (>=3.7)", "more_itertools", "more_itertools (>=8.8)", "packaging", "packaging (>=24.2)", "platformdirs (>=4.2.2)", "tomli (>=2.0.1) ; python_version < \"3.11\"", "wheel (>=0.43.0)"] cover = ["pytest-cov"] doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "pyproject-hooks (!=1.1)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier", "towncrier (<24.7)"] enabler = ["pytest-enabler (>=2.2)"] -test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.7.2)", "jaraco.test (>=5.5)", "packaging (>=24.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] -type = ["importlib_metadata (>=7.0.2)", "jaraco.develop (>=7.21)", "mypy (==1.14.*)", "pytest-mypy"] +test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21) ; python_version >= \"3.9\" and sys_platform != \"cygwin\"", "jaraco.envs (>=2.2)", "jaraco.path (>=3.7.2)", "jaraco.test (>=5.5)", "packaging (>=24.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf ; sys_platform != \"cygwin\"", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] +type = ["importlib_metadata (>=7.0.2) ; python_version < \"3.10\"", "jaraco.develop (>=7.21) ; sys_platform != \"cygwin\"", "mypy (==1.14.*)", "pytest-mypy"] [[package]] name = "six" @@ -4645,7 +4565,6 @@ description = "Python 2 and 3 compatibility utilities" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "six-1.17.0-py2.py3-none-any.whl", hash = "sha256:4721f391ed90541fddacab5acf947aa0d3dc7d27b2e1e8eda2be8970586c3274"}, {file = "six-1.17.0.tar.gz", hash = "sha256:ff70335d468e7eb6ec65b95b99d3a2836546063f63acc5171de367e834932a81"}, @@ -4658,7 +4577,6 @@ description = "Sniff out which async library your code is running under" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2"}, {file = "sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc"}, @@ -4671,7 +4589,7 @@ description = "A modern CSS selector implementation for Beautiful Soup." optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "soupsieve-2.6-py3-none-any.whl", hash = "sha256:e72c4ff06e4fb6e4b5a9f0f55fe6e81514581fca1515028625d0f299c602ccc9"}, {file = "soupsieve-2.6.tar.gz", hash = "sha256:e2e68417777af359ec65daac1057404a3c8a5455bb8abc36f1a9866ab1a51abb"}, @@ -4684,7 +4602,7 @@ description = "Database Abstraction Library" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"sql\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "extra == \"vector-db-based\" or extra == \"sql\"" files = [ {file = "SQLAlchemy-2.0.37-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:da36c3b0e891808a7542c5c89f224520b9a16c7f5e4d6a1156955605e54aef0e"}, {file = "SQLAlchemy-2.0.37-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:e7402ff96e2b073a98ef6d6142796426d705addd27b9d26c3b32dbaa06d7d069"}, @@ -4781,7 +4699,7 @@ description = "Pretty-print tabular data" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "tabulate-0.9.0-py3-none-any.whl", hash = "sha256:024ca478df22e9340661486f85298cff5f6dcdba14f3813e8830015b9ed1948f"}, {file = "tabulate-0.9.0.tar.gz", hash = "sha256:0095b12bf5966de529c0feb1fa08671671b3368eec77d7ef7ab114be2c068b3c"}, @@ -4797,7 +4715,6 @@ description = "Retry code until it succeeds" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "tenacity-8.5.0-py3-none-any.whl", hash = "sha256:b594c2a5945830c267ce6b79a166228323ed52718f30302c1359836112346687"}, {file = "tenacity-8.5.0.tar.gz", hash = "sha256:8bc6c0c8a09b31e6cad13c47afbed1a567518250a9a171418582ed8d9c20ca78"}, @@ -4814,7 +4731,6 @@ description = "Modern Text User Interface framework" optional = false python-versions = "<4.0.0,>=3.8.1" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "textual-1.0.0-py3-none-any.whl", hash = "sha256:2d4a701781c05104925e463ae370c630567c70c2880e92ab838052e3e23c986f"}, {file = "textual-1.0.0.tar.gz", hash = "sha256:bec9fe63547c1c552569d1b75d309038b7d456c03f86dfa3706ddb099b151399"}, @@ -4827,7 +4743,7 @@ rich = ">=13.3.3" typing-extensions = ">=4.4.0,<5.0.0" [package.extras] -syntax = ["tree-sitter (>=0.23.0)", "tree-sitter-bash (>=0.23.0)", "tree-sitter-css (>=0.23.0)", "tree-sitter-go (>=0.23.0)", "tree-sitter-html (>=0.23.0)", "tree-sitter-java (>=0.23.0)", "tree-sitter-javascript (>=0.23.0)", "tree-sitter-json (>=0.24.0)", "tree-sitter-markdown (>=0.3.0)", "tree-sitter-python (>=0.23.0)", "tree-sitter-regex (>=0.24.0)", "tree-sitter-rust (>=0.23.0)", "tree-sitter-sql (>=0.3.0)", "tree-sitter-toml (>=0.6.0)", "tree-sitter-xml (>=0.7.0)", "tree-sitter-yaml (>=0.6.0)"] +syntax = ["tree-sitter (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-bash (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-css (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-go (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-html (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-java (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-javascript (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-json (>=0.24.0) ; python_version >= \"3.9\"", "tree-sitter-markdown (>=0.3.0) ; python_version >= \"3.9\"", "tree-sitter-python (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-regex (>=0.24.0) ; python_version >= \"3.9\"", "tree-sitter-rust (>=0.23.0) ; python_version >= \"3.9\"", "tree-sitter-sql (>=0.3.0) ; python_version >= \"3.9\"", "tree-sitter-toml (>=0.6.0) ; python_version >= \"3.9\"", "tree-sitter-xml (>=0.7.0) ; python_version >= \"3.9\"", "tree-sitter-yaml (>=0.6.0) ; python_version >= \"3.9\""] [[package]] name = "threadpoolctl" @@ -4836,7 +4752,7 @@ description = "threadpoolctl" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "threadpoolctl-3.5.0-py3-none-any.whl", hash = "sha256:56c1e26c150397e58c4926da8eeee87533b1e32bef131bd4bf6a2f45f3185467"}, {file = "threadpoolctl-3.5.0.tar.gz", hash = "sha256:082433502dd922bf738de0d8bcc4fdcbf0979ff44c42bd40f5af8a282f6fa107"}, @@ -4849,7 +4765,7 @@ description = "tiktoken is a fast BPE tokeniser for use with OpenAI's models" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "tiktoken-0.8.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b07e33283463089c81ef1467180e3e00ab00d46c2c4bbcef0acab5f771d6695e"}, {file = "tiktoken-0.8.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9269348cb650726f44dd3bbb3f9110ac19a8dcc8f54949ad3ef652ca22a38e21"}, @@ -4898,7 +4814,6 @@ description = "A lil' TOML parser" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "tomli-2.2.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:678e4fa69e4575eb77d103de3df8a895e1591b48e740211bd1067378c69e8249"}, {file = "tomli-2.2.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:023aa114dd824ade0100497eb2318602af309e5a55595f76b626d6d9f3b7b0a6"}, @@ -4941,7 +4856,6 @@ description = "Fast, Extensible Progress Meter" optional = true python-versions = ">=3.7" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"}, {file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"}, @@ -4964,7 +4878,6 @@ description = "Typing stubs for cachetools" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "types-cachetools-5.5.0.20240820.tar.gz", hash = "sha256:b888ab5c1a48116f7799cd5004b18474cd82b5463acb5ffb2db2fc9c7b053bc0"}, {file = "types_cachetools-5.5.0.20240820-py3-none-any.whl", hash = "sha256:efb2ed8bf27a4b9d3ed70d33849f536362603a90b8090a328acf0cd42fda82e2"}, @@ -4977,7 +4890,6 @@ description = "Typing stubs for python-dateutil" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "types_python_dateutil-2.9.0.20241206-py3-none-any.whl", hash = "sha256:e248a4bc70a486d3e3ec84d0dc30eec3a5f979d6e7ee4123ae043eedbb987f53"}, {file = "types_python_dateutil-2.9.0.20241206.tar.gz", hash = "sha256:18f493414c26ffba692a72369fea7a154c502646301ebfe3d56a04b3767284cb"}, @@ -4990,7 +4902,7 @@ description = "Typing stubs for pytz" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "types_pytz-2024.2.0.20241221-py3-none-any.whl", hash = "sha256:8fc03195329c43637ed4f593663df721fef919b60a969066e22606edf0b53ad5"}, {file = "types_pytz-2024.2.0.20241221.tar.gz", hash = "sha256:06d7cde9613e9f7504766a0554a270c369434b50e00975b3a4a0f6eed0f2c1a9"}, @@ -5003,7 +4915,6 @@ description = "Typing stubs for PyYAML" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "types_PyYAML-6.0.12.20241230-py3-none-any.whl", hash = "sha256:fa4d32565219b68e6dee5f67534c722e53c00d1cfc09c435ef04d7353e1e96e6"}, {file = "types_pyyaml-6.0.12.20241230.tar.gz", hash = "sha256:7f07622dbd34bb9c8b264fe860a17e0efcad00d50b5f27e93984909d9363498c"}, @@ -5016,7 +4927,6 @@ description = "Typing stubs for requests" optional = false python-versions = ">=3.8" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "types-requests-2.32.0.20241016.tar.gz", hash = "sha256:0d9cad2f27515d0e3e3da7134a1b6f28fb97129d86b867f24d9c726452634d95"}, {file = "types_requests-2.32.0.20241016-py3-none-any.whl", hash = "sha256:4195d62d6d3e043a4eaaf08ff8a62184584d2e8684e9d2aa178c7915a7da3747"}, @@ -5032,7 +4942,6 @@ description = "Typing stubs for setuptools" optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "types_setuptools-75.8.2.20250305-py3-none-any.whl", hash = "sha256:ba80953fd1f5f49e552285c024f75b5223096a38a5138a54d18ddd3fa8f6a2d4"}, {file = "types_setuptools-75.8.2.20250305.tar.gz", hash = "sha256:a987269b49488f21961a1d99aa8d281b611625883def6392a93855b31544e405"}, @@ -5048,7 +4957,6 @@ description = "Backported and Experimental Type Hints for Python 3.8+" optional = false python-versions = ">=3.8" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "typing_extensions-4.12.2-py3-none-any.whl", hash = "sha256:04e5ca0351e0f3f85c6853954072df659d0d13fac324d0072316b67d7794700d"}, {file = "typing_extensions-4.12.2.tar.gz", hash = "sha256:1a7ead55c7e559dd4dee8856e3a88b41225abfe1ce8df57b7c13915fe121ffb8"}, @@ -5061,7 +4969,7 @@ description = "Runtime inspection utilities for typing module." optional = true python-versions = "*" groups = ["main"] -markers = "(extra == \"vector-db-based\" or extra == \"file-based\") and (python_version <= \"3.11\" or python_version >= \"3.12\")" +markers = "extra == \"vector-db-based\" or extra == \"file-based\"" files = [ {file = "typing_inspect-0.9.0-py3-none-any.whl", hash = "sha256:9ee6fc59062311ef8547596ab6b955e1b8aa46242d854bfc78f4f6b0eff35f9f"}, {file = "typing_inspect-0.9.0.tar.gz", hash = "sha256:b23fc42ff6f6ef6954e4852c1fb512cdd18dbea03134f91f856a95ccc9461f78"}, @@ -5078,7 +4986,6 @@ description = "Provider of IANA time zone data" optional = false python-versions = ">=2" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "tzdata-2025.1-py2.py3-none-any.whl", hash = "sha256:7e127113816800496f027041c570f50bcd464a020098a3b6b199517772303639"}, {file = "tzdata-2025.1.tar.gz", hash = "sha256:24894909e88cdb28bd1636c6887801df64cb485bd593f2fd83ef29075a81d694"}, @@ -5091,7 +4998,6 @@ description = "Micro subset of unicode data files for linkify-it-py projects." optional = false python-versions = ">=3.7" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "uc-micro-py-1.0.3.tar.gz", hash = "sha256:d321b92cff673ec58027c04015fcaa8bb1e005478643ff4a500882eaab88c48a"}, {file = "uc_micro_py-1.0.3-py3-none-any.whl", hash = "sha256:db1dffff340817673d7b466ec86114a9dc0e9d4d9b5ba229d9d60e5c12600cd5"}, @@ -5107,7 +5013,7 @@ description = "A library that prepares raw documents for downstream ML tasks." optional = true python-versions = ">=3.7.0" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "unstructured-0.10.27-py3-none-any.whl", hash = "sha256:3a8a8e44302388ddc39c184059e8b4458f1cdc58032540b9af7d85f6c3eca3be"}, {file = "unstructured-0.10.27.tar.gz", hash = "sha256:f567b5c4385993a9ab48db5563dd7b413aac4f2002bb22e6250496ea8f440f5e"}, @@ -5189,7 +5095,7 @@ description = "Python-tesseract is a python wrapper for Google's Tesseract-OCR" optional = true python-versions = ">=3.8" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "unstructured.pytesseract-0.3.13-py3-none-any.whl", hash = "sha256:8001bc860470d56185176eb3ceb4623e888eba058ca3b30af79003784bc40e19"}, {file = "unstructured.pytesseract-0.3.13.tar.gz", hash = "sha256:ff2e6391496e457dbf4b4e327f4a4577cce18921ea6570dc74bd64381b10e963"}, @@ -5206,7 +5112,6 @@ description = "URL normalization for Python" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "url-normalize-1.4.3.tar.gz", hash = "sha256:d23d3a070ac52a67b83a1c59a0e68f8608d1cd538783b401bc9de2c0fac999b2"}, {file = "url_normalize-1.4.3-py2.py3-none-any.whl", hash = "sha256:ec3c301f04e5bb676d333a7fa162fa977ad2ca04b7e652bfc9fac4e405728eed"}, @@ -5222,14 +5127,13 @@ description = "HTTP library with thread-safe connection pooling, file post, and optional = false python-versions = ">=3.9" groups = ["main", "dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "urllib3-2.3.0-py3-none-any.whl", hash = "sha256:1cee9ad369867bfdbbb48b7dd50374c0967a0bb7710050facf0dd6911440e3df"}, {file = "urllib3-2.3.0.tar.gz", hash = "sha256:f8c5449b3cf0861679ce7e0503c7b44b5ec981bec0d1d3795a07f1ba96f0204d"}, ] [package.extras] -brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)"] +brotli = ["brotli (>=1.0.9) ; platform_python_implementation == \"CPython\"", "brotlicffi (>=0.8.0) ; platform_python_implementation != \"CPython\""] h2 = ["h2 (>=4,<5)"] socks = ["pysocks (>=1.5.6,!=1.5.7,<2.0)"] zstd = ["zstandard (>=0.18.0)"] @@ -5241,7 +5145,6 @@ description = "Wildcard/glob file name matcher." optional = false python-versions = ">=3.8" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "wcmatch-10.0-py3-none-any.whl", hash = "sha256:0dd927072d03c0a6527a20d2e6ad5ba8d0380e60870c383bc533b71744df7b7a"}, {file = "wcmatch-10.0.tar.gz", hash = "sha256:e72f0de09bba6a04e0de70937b0cf06e55f36f37b3deb422dfaf854b867b840a"}, @@ -5257,7 +5160,6 @@ description = "The comprehensive WSGI web application library." optional = false python-versions = ">=3.9" groups = ["dev"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "werkzeug-3.1.3-py3-none-any.whl", hash = "sha256:54b78bf3716d19a65be4fceccc0d1d7b89e608834989dfae50ea87564639213e"}, {file = "werkzeug-3.1.3.tar.gz", hash = "sha256:60723ce945c19328679790e3282cc758aa4a6040e4bb330f53d30fa546d44746"}, @@ -5276,7 +5178,6 @@ description = "Modern datetime library for Python" optional = false python-versions = ">=3.9" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "whenever-0.6.16-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:901783ba877b5d73ce5b1bc1697c6097a9ac14c43064788b24ec7dc75a85a90a"}, {file = "whenever-0.6.16-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:4d374cd750ea68adb4ad69d52aef3838eda38ae63183c6135b122772ac053c66"}, @@ -5361,7 +5262,7 @@ description = "A Python module for creating Excel XLSX files." optional = true python-versions = ">=3.6" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"file-based\"" +markers = "extra == \"file-based\"" files = [ {file = "XlsxWriter-3.2.0-py3-none-any.whl", hash = "sha256:ecfd5405b3e0e228219bcaf24c2ca0915e012ca9464a14048021d21a995d490e"}, {file = "XlsxWriter-3.2.0.tar.gz", hash = "sha256:9977d0c661a72866a61f9f7a809e25ebbb0fb7036baa3b9fe74afcfca6b3cb8c"}, @@ -5374,7 +5275,6 @@ description = "Makes working with XML feel like you are working with JSON" optional = false python-versions = ">=3.6" groups = ["main"] -markers = "python_version <= \"3.11\" or python_version >= \"3.12\"" files = [ {file = "xmltodict-0.14.2-py2.py3-none-any.whl", hash = "sha256:20cc7d723ed729276e808f26fb6b3599f786cbc37e06c65e192ba77c40f20aac"}, {file = "xmltodict-0.14.2.tar.gz", hash = "sha256:201e7c28bb210e374999d1dde6382923ab0ed1a8a5faeece48ab525b7810a553"}, @@ -5387,7 +5287,7 @@ description = "Yet another URL library" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "yarl-1.18.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7df647e8edd71f000a5208fe6ff8c382a1de8edfbccdbbfe649d263de07d8c34"}, {file = "yarl-1.18.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c69697d3adff5aa4f874b19c0e4ed65180ceed6318ec856ebc423aa5850d84f7"}, @@ -5485,18 +5385,18 @@ description = "Backport of pathlib-compatible object wrapper for zip files" optional = true python-versions = ">=3.9" groups = ["main"] -markers = "(python_version <= \"3.11\" or python_version >= \"3.12\") and extra == \"vector-db-based\"" +markers = "extra == \"vector-db-based\"" files = [ {file = "zipp-3.21.0-py3-none-any.whl", hash = "sha256:ac1bbe05fd2991f160ebce24ffbac5f6d11d83dc90891255885223d42b3cd931"}, {file = "zipp-3.21.0.tar.gz", hash = "sha256:2c9958f6430a2040341a52eb608ed6dd93ef4392e02ffe219417c1b28b5dd1f4"}, ] [package.extras] -check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\""] cover = ["pytest-cov"] doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] enabler = ["pytest-enabler (>=2.2)"] -test = ["big-O", "importlib-resources", "jaraco.functools", "jaraco.itertools", "jaraco.test", "more-itertools", "pytest (>=6,!=8.1.*)", "pytest-ignore-flaky"] +test = ["big-O", "importlib-resources ; python_version < \"3.9\"", "jaraco.functools", "jaraco.itertools", "jaraco.test", "more-itertools", "pytest (>=6,!=8.1.*)", "pytest-ignore-flaky"] type = ["pytest-mypy"] [extras] @@ -5507,4 +5407,4 @@ vector-db-based = ["cohere", "langchain", "openai", "tiktoken"] [metadata] lock-version = "2.1" python-versions = ">=3.10,<3.13" -content-hash = "c8731f26643e07136e524d5e0d6e0f5c2229cf63d43bf5644de9f1cf8e565197" +content-hash = "b581ab987c1608518a0bb26fcda2bdbda9b245940d7cbe397f2092e7c3a73efb" diff --git a/pyproject.toml b/pyproject.toml index d236c0b9d..aa1151c3d 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -30,7 +30,8 @@ enable = true [tool.poetry.dependencies] python = ">=3.10,<3.13" -airbyte-protocol-models-dataclasses = "^0.14" +airbyte-protocol-models-dataclasses = { version = "0.14.1337.dev1742858109", source = "testpypi" } + backoff = "*" cachetools = "*" dpath = "^2.1.6" @@ -84,6 +85,11 @@ xmltodict = ">=0.13,<0.15" anyascii = "^0.3.2" whenever = "^0.6.16" +[[tool.poetry.source]] +name = "testpypi" +url = "https://test.pypi.org/simple/" +priority = "supplemental" + [tool.poetry.group.dev.dependencies] freezegun = "*" mypy = "*" diff --git a/unit_tests/sources/declarative/file/test_file_stream.py b/unit_tests/sources/declarative/file/test_file_stream.py index 468ca086e..62b05a8db 100644 --- a/unit_tests/sources/declarative/file/test_file_stream.py +++ b/unit_tests/sources/declarative/file/test_file_stream.py @@ -1,3 +1,4 @@ +import re from pathlib import Path from typing import Any, Dict, List, Optional from unittest import TestCase @@ -7,6 +8,7 @@ from airbyte_cdk.sources.declarative.yaml_declarative_source import YamlDeclarativeSource from airbyte_cdk.test.catalog_builder import CatalogBuilder, ConfiguredAirbyteStreamBuilder from airbyte_cdk.test.entrypoint_wrapper import EntrypointOutput +from airbyte_cdk.test.entrypoint_wrapper import discover as entrypoint_discover from airbyte_cdk.test.entrypoint_wrapper import read as entrypoint_read from airbyte_cdk.test.state_builder import StateBuilder @@ -19,13 +21,25 @@ def build(self) -> Dict[str, Any]: "credentials": { "credentials": "api_token", "email": "integration-test@airbyte.io", - "api_token": - } + "api_token": "fake token", + }, } -def _source(catalog: ConfiguredAirbyteCatalog, config: Dict[str, Any], state: Optional[List[AirbyteStateMessage]] = None) -> YamlDeclarativeSource: - return YamlDeclarativeSource(path_to_yaml=str(Path(__file__).parent / "file_stream_manifest.yaml"), catalog=catalog, config=config, state=state) +def _source( + catalog: ConfiguredAirbyteCatalog, + config: Dict[str, Any], + state: Optional[List[AirbyteStateMessage]] = None, + yaml_file: Optional[str] = None, +) -> YamlDeclarativeSource: + if not yaml_file: + yaml_file = "file_stream_manifest.yaml" + return YamlDeclarativeSource( + path_to_yaml=str(Path(__file__).parent / yaml_file), + catalog=catalog, + config=config, + state=state, + ) def read( @@ -33,10 +47,20 @@ def read( catalog: ConfiguredAirbyteCatalog, state_builder: Optional[StateBuilder] = None, expecting_exception: bool = False, + yaml_file: Optional[str] = None, ) -> EntrypointOutput: config = config_builder.build() state = state_builder.build() if state_builder else StateBuilder().build() - return entrypoint_read(_source(catalog, config, state), config, catalog, state, expecting_exception) + return entrypoint_read( + _source(catalog, config, state, yaml_file), config, catalog, state, expecting_exception + ) + + +def discover(config_builder: ConfigBuilder, expecting_exception: bool = False) -> EntrypointOutput: + config = config_builder.build() + return entrypoint_discover( + _source(CatalogBuilder().build(), config), config, expecting_exception + ) class FileStreamTest(TestCase): @@ -44,18 +68,73 @@ def _config(self) -> ConfigBuilder: return ConfigBuilder() def test_check(self) -> None: - source = _source(CatalogBuilder().with_stream(ConfiguredAirbyteStreamBuilder().with_name("articles")).build(), self._config().build()) + source = _source( + CatalogBuilder() + .with_stream(ConfiguredAirbyteStreamBuilder().with_name("articles")) + .build(), + self._config().build(), + ) check_result = source.check(Mock(), self._config().build()) assert check_result.status == Status.SUCCEEDED def test_get_articles(self) -> None: - output = read(self._config(), CatalogBuilder().with_stream(ConfiguredAirbyteStreamBuilder().with_name("articles")).build()) + output = read( + self._config(), + CatalogBuilder() + .with_stream(ConfiguredAirbyteStreamBuilder().with_name("articles")) + .build(), + ) assert output.records def test_get_article_attachments(self) -> None: - output = read(self._config(), CatalogBuilder().with_stream(ConfiguredAirbyteStreamBuilder().with_name("article_attachments")).build()) + output = read( + self._config(), + CatalogBuilder() + .with_stream(ConfiguredAirbyteStreamBuilder().with_name("article_attachments")) + .build(), + ) + + assert output.records + file_reference = output.records[0].record.file_reference + assert file_reference + assert file_reference.file_url + assert re.match(r"^.*/article_attachments/[0-9a-fA-F-]{36}$", file_reference.file_url) + assert file_reference.file_relative_path + assert re.match( + r"^article_attachments/[0-9a-fA-F-]{36}$", file_reference.file_relative_path + ) + assert file_reference.file_size_bytes + + def test_get_article_attachments_with_filename_extractor(self) -> None: + output = read( + self._config(), + CatalogBuilder() + .with_stream(ConfiguredAirbyteStreamBuilder().with_name("article_attachments")) + .build(), + yaml_file="test_file_stream_with_filename_extractor.yaml", + ) assert output.records + file_reference = output.records[0].record.file_reference + assert file_reference + assert file_reference.file_url + # todo: once we finally mock the response update to check file name + assert not re.match(r"^.*/article_attachments/[0-9a-fA-F-]{36}$", file_reference.file_url) + assert file_reference.file_relative_path + assert not re.match( + r"^article_attachments/[0-9a-fA-F-]{36}$", file_reference.file_relative_path + ) + assert file_reference.file_size_bytes + + def test_discover_article_attachments(self) -> None: + output = discover(self._config()) + + article_attachments_stream = next( + filter( + lambda stream: stream.name == "article_attachments", output.catalog.catalog.streams + ) + ) + assert article_attachments_stream.is_file_based diff --git a/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml b/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml new file mode 100644 index 000000000..d827189ab --- /dev/null +++ b/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml @@ -0,0 +1,191 @@ +version: 2.0.0 + +type: DeclarativeSource + +check: + type: CheckStream + stream_names: + - "articles" + +definitions: + bearer_authenticator: + type: BearerAuthenticator + api_token: "{{ config['credentials']['access_token'] }}" + basic_authenticator: + type: BasicHttpAuthenticator + username: "{{ config['credentials']['email'] + '/token' }}" + password: "{{ config['credentials']['api_token'] }}" + + retriever: + type: SimpleRetriever + requester: + type: HttpRequester + url_base: https://{{ config['subdomain'] }}.zendesk.com/api/v2/ + http_method: GET + authenticator: + type: SelectiveAuthenticator + authenticator_selection_path: ["credentials", "credentials"] + authenticators: + oauth2.0: "#/definitions/bearer_authenticator" + api_token: "#/definitions/basic_authenticator" + record_selector: + type: RecordSelector + extractor: + type: DpathExtractor + field_path: + ["{{ parameters.get('data_path') or parameters.get('name') }}"] + schema_normalization: Default + paginator: + type: DefaultPaginator + page_size_option: + type: RequestOption + field_name: "per_page" + inject_into: request_parameter + pagination_strategy: + type: CursorPagination + page_size: 100 + cursor_value: '{{ response.get("next_page", {}) }}' + stop_condition: "{{ last_page_size == 0 }}" + page_token_option: + type: RequestPath + + base_stream: + type: DeclarativeStream + schema_loader: + type: JsonFileSchemaLoader + retriever: + $ref: "#/definitions/retriever" + + cursor_incremental_sync: + type: DatetimeBasedCursor + cursor_datetime_formats: + - "%s" + - "%Y-%m-%dT%H:%M:%SZ" + - "%Y-%m-%dT%H:%M:%S%z" + datetime_format: "%s" + cursor_field: "{{ parameters.get('cursor_field', 'updated_at') }}" + start_datetime: + datetime: "{{ timestamp(config.get('start_date')) | int if config.get('start_date') else day_delta(-730, '%s') }}" + start_time_option: + inject_into: request_parameter + field_name: "{{ parameters['cursor_filter'] }}" + type: RequestOption + + base_incremental_stream: + $ref: "#/definitions/base_stream" + incremental_sync: + $ref: "#/definitions/cursor_incremental_sync" + + # Incremental cursor-based streams + articles_stream: + $ref: "#/definitions/base_incremental_stream" + name: "articles" + primary_key: "id" + schema_loader: + type: InlineSchemaLoader + schema: + type: object + $schema: http://json-schema.org/schema# + properties: + id: + type: integer + additionalProperties: true + incremental_sync: + $ref: "#/definitions/cursor_incremental_sync" + start_time_option: + $ref: "#/definitions/cursor_incremental_sync/start_time_option" + field_name: "start_time" + retriever: + $ref: "#/definitions/retriever" + ignore_stream_slicer_parameters_on_paginated_requests: true + requester: + $ref: "#/definitions/retriever/requester" + path: "help_center/incremental/articles" + paginator: + type: DefaultPaginator + pagination_strategy: + type: CursorPagination + cursor_value: '{{ response.get("next_page", {}) }}' + stop_condition: "{{ config.get('ignore_pagination', False) or last_page_size == 0 }}" + page_token_option: + type: RequestPath + record_selector: + extractor: + type: DpathExtractor + field_path: ["articles"] + + article_attachments_stream: + $ref: "#/definitions/base_incremental_stream" + name: "article_attachments" + primary_key: "id" + schema_loader: + type: InlineSchemaLoader + schema: + type: object + $schema: http://json-schema.org/schema# + properties: + id: + type: integer + additionalProperties: true + retriever: + $ref: "#/definitions/retriever" + ignore_stream_slicer_parameters_on_paginated_requests: true + requester: + $ref: "#/definitions/retriever/requester" + path: "help_center/articles/{{ stream_partition.article_id }}/attachments" + partition_router: + type: SubstreamPartitionRouter + parent_stream_configs: + - type: ParentStreamConfig + parent_key: "id" + partition_field: "article_id" + stream: + $ref: "#/definitions/articles_stream" + incremental_dependency: true + record_selector: + extractor: + type: DpathExtractor + field_path: ["article_attachments"] + file_uploader: + type: FileUploader + requester: + type: HttpRequester + url_base: "{{download_target}}" + http_method: GET + authenticator: + type: SelectiveAuthenticator + authenticator_selection_path: [ "credentials", "credentials" ] + authenticators: + oauth2.0: "#/definitions/bearer_authenticator" + api_token: "#/definitions/basic_authenticator" + download_target_extractor: + type: DpathExtractor + field_path: [ "content_url" ] + filename_extractor: "{{ record.id }}/{{ record.file_name }}/" + + +streams: + - $ref: "#/definitions/articles_stream" + - $ref: "#/definitions/article_attachments_stream" + +spec: + type: Spec + connection_specification: + type: object + $schema: http://json-schema.org/draft-07/schema# + required: + - subdomain + - start_date + properties: + subdomain: + type: string + name: subdomain + order: 0 + title: Subdomain + start_date: + type: string + order: 1 + title: Start date + format: date-time + pattern: ^[0-9]{4}-[0-9]{2}-[0-9]{2}T[0-9]{2}:[0-9]{2}:[0-9]{2}Z$ + additionalProperties: true