From d326a2689b49612479086908e748cf6a06ad3bc2 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Tue, 3 Dec 2024 16:24:05 +0200 Subject: [PATCH 01/34] Add concurrent PerPartitionCursor --- .../concurrent_declarative_source.py | 58 ++++ .../declarative/incremental/__init__.py | 3 + .../concurrent_partition_cursor.py | 309 ++++++++++++++++++ .../incremental/per_partition_cursor.py | 9 + .../parsers/model_to_component_factory.py | 176 ++++++++++ .../declarative_partition_generator.py | 26 ++ .../sources/streams/concurrent/cursor.py | 9 + 7 files changed, 590 insertions(+) create mode 100644 airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 001740a35..163912da5 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -3,6 +3,8 @@ # import logging +from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import PerPartitionCursor +from airbyte_cdk.sources.declarative.incremental.per_partition_with_global import PerPartitionWithGlobalCursor from typing import Any, Generic, Iterator, List, Mapping, Optional, Tuple, Union, Callable from airbyte_cdk.models import ( @@ -37,6 +39,7 @@ from airbyte_cdk.sources.declarative.retrievers import SimpleRetriever, Retriever from airbyte_cdk.sources.declarative.stream_slicers.declarative_partition_generator import ( DeclarativePartitionFactory, + DeclarativePartitionFactory1, StreamSlicerPartitionGenerator, ) from airbyte_cdk.sources.declarative.transformations.add_fields import AddFields @@ -219,6 +222,61 @@ def _group_streams( ) ) + partition_generator = StreamSlicerPartitionGenerator( + DeclarativePartitionFactory( + declarative_stream.name, + declarative_stream.get_json_schema(), + self._retriever_factory( + name_to_stream_mapping[declarative_stream.name], + config, + stream_state, + ), + self.message_repository, + ), + cursor, + ) + + concurrent_streams.append( + DefaultStream( + partition_generator=partition_generator, + name=declarative_stream.name, + json_schema=declarative_stream.get_json_schema(), + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=get_primary_key_from_stream(declarative_stream.primary_key), + cursor_field=cursor.cursor_field.cursor_field_key, + logger=self.logger, + cursor=cursor, + ) + ) + elif ( + datetime_based_cursor_component_definition + and datetime_based_cursor_component_definition.get("type", "") + == DatetimeBasedCursorModel.__name__ + and self._stream_supports_concurrent_partition_processing( + declarative_stream=declarative_stream + ) + and hasattr(declarative_stream.retriever, "stream_slicer") + and isinstance(declarative_stream.retriever.stream_slicer, PerPartitionWithGlobalCursor) + ): + stream_state = state_manager.get_stream_state( + stream_name=declarative_stream.name, namespace=declarative_stream.namespace + ) + partition_router = declarative_stream.retriever.stream_slicer._partition_router + + cursor, connector_state_converter = ( + self._constructor.create_concurrent_cursor_from_perpartition_cursor( + state_manager=state_manager, + model_type=DatetimeBasedCursorModel, + component_definition=datetime_based_cursor_component_definition, + stream_name=declarative_stream.name, + stream_namespace=declarative_stream.namespace, + config=config or {}, + stream_state=stream_state, + partition_router=partition_router, + ) + ) + + partition_generator = StreamSlicerPartitionGenerator( DeclarativePartitionFactory( declarative_stream.name, diff --git a/airbyte_cdk/sources/declarative/incremental/__init__.py b/airbyte_cdk/sources/declarative/incremental/__init__.py index 11c1cba99..7acd681ec 100644 --- a/airbyte_cdk/sources/declarative/incremental/__init__.py +++ b/airbyte_cdk/sources/declarative/incremental/__init__.py @@ -2,6 +2,7 @@ # Copyright (c) 2022 Airbyte, Inc., all rights reserved. # +from airbyte_cdk.sources.declarative.incremental.concurrent_partition_cursor import ConcurrentCursorFactory, ConcurrentPerPartitionCursor from airbyte_cdk.sources.declarative.incremental.datetime_based_cursor import DatetimeBasedCursor from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import GlobalSubstreamCursor @@ -14,6 +15,8 @@ __all__ = [ "CursorFactory", + "ConcurrentCursorFactory" + "ConcurrentPerPartitionCursor", "DatetimeBasedCursor", "DeclarativeCursor", "GlobalSubstreamCursor", diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py new file mode 100644 index 000000000..74e8ff5f6 --- /dev/null +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -0,0 +1,309 @@ +import copy + +from airbyte_cdk.sources.streams.concurrent.cursor import Cursor + +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +from collections import OrderedDict +from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional, Union + +from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor +from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter +from airbyte_cdk.sources.streams.checkpoint.per_partition_key_serializer import ( + PerPartitionKeySerializer, +) +from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, CursorField, CursorValueType, GapType +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.types import Record, StreamSlice, StreamState +import functools +from abc import ABC, abstractmethod +from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional, Protocol, Tuple + +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.message import MessageRepository +from airbyte_cdk.sources.streams import NO_CURSOR_STATE_KEY +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.streams.concurrent.partitions.record import Record +from airbyte_cdk.sources.streams.concurrent.partitions.stream_slicer import StreamSlicer +from airbyte_cdk.sources.streams.concurrent.state_converters.abstract_stream_state_converter import ( + AbstractStreamStateConverter, +) +from airbyte_cdk.sources.types import StreamSlice + +logger = logging.getLogger("airbyte") + + +class ConcurrentCursorFactory: + def __init__(self, create_function: Callable[[Mapping[str, Any]], DeclarativeCursor]): + self._create_function = create_function + + def create(self, stream_state: Mapping[str, Any]) -> DeclarativeCursor: + return self._create_function(stream_state=stream_state)[0] + + +class ConcurrentPerPartitionCursor(Cursor): + """ + Manages state per partition when a stream has many partitions, to prevent data loss or duplication. + + **Partition Limitation and Limit Reached Logic** + + - **DEFAULT_MAX_PARTITIONS_NUMBER**: The maximum number of partitions to keep in memory (default is 10,000). + - **_cursor_per_partition**: An ordered dictionary that stores cursors for each partition. + - **_over_limit**: A counter that increments each time an oldest partition is removed when the limit is exceeded. + + The class ensures that the number of partitions tracked does not exceed the `DEFAULT_MAX_PARTITIONS_NUMBER` to prevent excessive memory usage. + + - When the number of partitions exceeds the limit, the oldest partitions are removed from `_cursor_per_partition`, and `_over_limit` is incremented accordingly. + - The `limit_reached` method returns `True` when `_over_limit` exceeds `DEFAULT_MAX_PARTITIONS_NUMBER`, indicating that the global cursor should be used instead of per-partition cursors. + + This approach avoids unnecessary switching to a global cursor due to temporary spikes in partition counts, ensuring that switching is only done when a sustained high number of partitions is observed. + """ + + DEFAULT_MAX_PARTITIONS_NUMBER = 10000 + _NO_STATE: Mapping[str, Any] = {} + _NO_CURSOR_STATE: Mapping[str, Any] = {} + _KEY = 0 + _VALUE = 1 + _state_to_migrate_from: Mapping[str, Any] = {} + + def __init__( + self, + cursor_factory: ConcurrentCursorFactory, + partition_router: PartitionRouter, + stream_name: str, + stream_namespace: Optional[str], + stream_state: Any, + message_repository: MessageRepository, + connector_state_manager: ConnectorStateManager, + connector_state_converter: AbstractStreamStateConverter, + cursor_field: CursorField, + slice_boundary_fields: Optional[Tuple[str, str]], + start: Optional[CursorValueType], + end_provider: Callable[[], CursorValueType], + lookback_window: Optional[GapType] = None, + slice_range: Optional[GapType] = None, + cursor_granularity: Optional[GapType] = None, + ) -> None: + self._stream_name = stream_name + self._stream_namespace = stream_namespace + self._message_repository = message_repository + self._connector_state_converter = connector_state_converter + self._connector_state_manager = connector_state_manager + self._cursor_field = cursor_field + # To see some example where the slice boundaries might not be defined, check https://github.com/airbytehq/airbyte/blob/1ce84d6396e446e1ac2377362446e3fb94509461/airbyte-integrations/connectors/source-stripe/source_stripe/streams.py#L363-L379 + self._slice_boundary_fields = slice_boundary_fields + self._start = start + self._end_provider = end_provider + self._conccurent_state = stream_state + # self.start, self._concurrent_state = self._get_concurrent_state(stream_state) + self._lookback_window = lookback_window + self._slice_range = slice_range + self._most_recent_cursor_value_per_partition: MutableMapping[Partition, Any] = {} + self._has_closed_at_least_one_slice = False + self._cursor_granularity = cursor_granularity + + self._cursor_factory = cursor_factory + self._partition_router = partition_router + # The dict is ordered to ensure that once the maximum number of partitions is reached, + # the oldest partitions can be efficiently removed, maintaining the most recent partitions. + self._cursor_per_partition: OrderedDict[str, Cursor] = OrderedDict() + self._over_limit = 0 + self._state = {} + self._partition_serializer = PerPartitionKeySerializer() + + self._set_initial_state(stream_state) + + @property + def state(self) -> MutableMapping[str, Any]: + states = [] + for partition_tuple, cursor in self._cursor_per_partition.items(): + cursor_state = cursor._connector_state_converter.convert_to_state_message( + cursor._cursor_field, cursor.state + ) + # print(cursor_state, cursor.state) + if cursor_state: + states.append( + { + "partition": self._to_dict(partition_tuple), + "cursor": copy.deepcopy(cursor_state), + } + ) + state: dict[str, Any] = {"states": states} + # print(state) + return state + + @property + def cursor_field(self) -> CursorField: + return self._cursor_field + + def close_partition(self, partition: Partition) -> None: + self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)].close_partition_without_emit(partition=partition) + + def ensure_at_least_one_state_emitted(self) -> None: + """ + The platform expect to have at least one state message on successful syncs. Hence, whatever happens, we expect this method to be + called. + """ + self._emit_state_message() + + def _emit_state_message(self) -> None: + self._connector_state_manager.update_state_for_stream( + self._stream_name, + self._stream_namespace, + self.state, + ) + state_message = self._connector_state_manager.create_state_message( + self._stream_name, self._stream_namespace + ) + self._message_repository.emit_message(state_message) + + + def stream_slices(self) -> Iterable[StreamSlice]: + slices = self._partition_router.stream_slices() + for partition in slices: + yield from self.generate_slices_from_partition(partition) + + def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[StreamSlice]: + # Ensure the maximum number of partitions is not exceeded + self._ensure_partition_limit() + + cursor = self._cursor_per_partition.get(self._to_partition_key(partition.partition)) + if not cursor: + partition_state = ( + self._state_to_migrate_from + if self._state_to_migrate_from + else self._NO_CURSOR_STATE + ) + cursor = self._create_cursor(partition_state) + self._cursor_per_partition[self._to_partition_key(partition.partition)] = cursor + + for cursor_slice in cursor.stream_slices(): + yield StreamSlice( + partition=partition, cursor_slice=cursor_slice, extra_fields=partition.extra_fields + ) + + def _ensure_partition_limit(self) -> None: + """ + Ensure the maximum number of partitions is not exceeded. If so, the oldest added partition will be dropped. + """ + while len(self._cursor_per_partition) > self.DEFAULT_MAX_PARTITIONS_NUMBER - 1: + self._over_limit += 1 + oldest_partition = self._cursor_per_partition.popitem(last=False)[ + 0 + ] # Remove the oldest partition + logger.warning( + f"The maximum number of partitions has been reached. Dropping the oldest partition: {oldest_partition}. Over limit: {self._over_limit}." + ) + + def limit_reached(self) -> bool: + return self._over_limit > self.DEFAULT_MAX_PARTITIONS_NUMBER + + def _set_initial_state(self, stream_state: StreamState) -> None: + """ + Set the initial state for the cursors. + + This method initializes the state for each partition cursor using the provided stream state. + If a partition state is provided in the stream state, it will update the corresponding partition cursor with this state. + + Additionally, it sets the parent state for partition routers that are based on parent streams. If a partition router + does not have parent streams, this step will be skipped due to the default PartitionRouter implementation. + + Args: + stream_state (StreamState): The state of the streams to be set. The format of the stream state should be: + { + "states": [ + { + "partition": { + "partition_key": "value" + }, + "cursor": { + "last_updated": "2023-05-27T00:00:00Z" + } + } + ], + "parent_state": { + "parent_stream_name": { + "last_updated": "2023-05-27T00:00:00Z" + } + } + } + """ + if not stream_state: + return + + if "states" not in stream_state: + # We assume that `stream_state` is in a global format that can be applied to all partitions. + # Example: {"global_state_format_key": "global_state_format_value"} + self._state_to_migrate_from = stream_state + + else: + for state in stream_state["states"]: + self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( + self._create_cursor(state["cursor"]) + ) + + # set default state for missing partitions if it is per partition with fallback to global + if "state" in stream_state: + self._state_to_migrate_from = stream_state["state"] + + # Set parent state for partition routers based on parent streams + self._partition_router.set_initial_state(stream_state) + + def observe(self, record: Record) -> None: + print(f"ESTATE: {self._to_partition_key(record.partition._stream_slice.partition)}: {record.data[self.cursor_field.cursor_field_key]}") + self._cursor_per_partition[self._to_partition_key(record.partition._stream_slice.partition)].observe(record) + + def _to_partition_key(self, partition: Mapping[str, Any]) -> str: + return self._partition_serializer.to_partition_key(partition) + + def _to_dict(self, partition_key: str) -> Mapping[str, Any]: + return self._partition_serializer.to_partition(partition_key) + + def _create_cursor(self, cursor_state: Any) -> DeclarativeCursor: + cursor = self._cursor_factory.create(stream_state=cursor_state) + return cursor + + def should_be_synced(self, record: Record) -> bool: + return self._get_cursor(record).should_be_synced( + self._convert_record_to_cursor_record(record) + ) + + def is_greater_than_or_equal(self, first: Record, second: Record) -> bool: + if not first.associated_slice or not second.associated_slice: + raise ValueError( + f"Both records should have an associated slice but got {first.associated_slice} and {second.associated_slice}" + ) + if first.associated_slice.partition != second.associated_slice.partition: + raise ValueError( + f"To compare records, partition should be the same but got {first.associated_slice.partition} and {second.associated_slice.partition}" + ) + + return self._get_cursor(first).is_greater_than_or_equal( + self._convert_record_to_cursor_record(first), + self._convert_record_to_cursor_record(second), + ) + + @staticmethod + def _convert_record_to_cursor_record(record: Record) -> Record: + return Record( + record.data, + StreamSlice(partition={}, cursor_slice=record.associated_slice.cursor_slice) + if record.associated_slice + else None, + ) + + def _get_cursor(self, record: Record) -> DeclarativeCursor: + if not record.associated_slice: + raise ValueError( + "Invalid state as stream slices that are emitted should refer to an existing cursor" + ) + partition_key = self._to_partition_key(record.associated_slice.partition) + if partition_key not in self._cursor_per_partition: + raise ValueError( + "Invalid state as stream slices that are emitted should refer to an existing cursor" + ) + cursor = self._cursor_per_partition[partition_key] + return cursor diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py index a6449d811..c8dd0e2f6 100644 --- a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py @@ -244,6 +244,15 @@ def get_request_headers( next_page_token: Optional[Mapping[str, Any]] = None, ) -> Mapping[str, Any]: if stream_slice: + if self._to_partition_key(stream_slice.partition) not in self._cursor_per_partition: + partition_state = ( + self._state_to_migrate_from + if self._state_to_migrate_from + else self._NO_CURSOR_STATE + ) + cursor = self._create_cursor(partition_state) + + self._cursor_per_partition[self._to_partition_key(stream_slice.partition)] = cursor return self._partition_router.get_request_headers( # type: ignore # this always returns a mapping stream_state=stream_state, stream_slice=StreamSlice(partition=stream_slice.partition, cursor_slice={}), 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 d2dd9d9dc..ad9fc9ccc 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -79,6 +79,8 @@ ) from airbyte_cdk.sources.declarative.incremental import ( ChildPartitionResumableFullRefreshCursor, + ConcurrentPerPartitionCursor, + ConcurrentCursorFactory, CursorFactory, DatetimeBasedCursor, DeclarativeCursor, @@ -902,6 +904,180 @@ def create_concurrent_cursor_from_datetime_based_cursor( ), connector_state_converter, ) + def create_concurrent_cursor_from_perpartition_cursor( + self, + state_manager: ConnectorStateManager, + model_type: Type[BaseModel], + component_definition: ComponentDefinition, + stream_name: str, + stream_namespace: Optional[str], + config: Config, + stream_state: MutableMapping[str, Any], + partition_router, + **kwargs: Any, + ) -> Tuple[ConcurrentCursor, DateTimeStreamStateConverter]: + component_type = component_definition.get("type") + if component_definition.get("type") != model_type.__name__: + raise ValueError( + f"Expected manifest component of type {model_type.__name__}, but received {component_type} instead" + ) + + datetime_based_cursor_model = model_type.parse_obj(component_definition) + + if not isinstance(datetime_based_cursor_model, DatetimeBasedCursorModel): + raise ValueError( + f"Expected {model_type.__name__} component, but received {datetime_based_cursor_model.__class__.__name__}" + ) + + interpolated_cursor_field = InterpolatedString.create( + datetime_based_cursor_model.cursor_field, + parameters=datetime_based_cursor_model.parameters or {}, + ) + cursor_field = CursorField(interpolated_cursor_field.eval(config=config)) + + interpolated_partition_field_start = InterpolatedString.create( + datetime_based_cursor_model.partition_field_start or "start_time", + parameters=datetime_based_cursor_model.parameters or {}, + ) + interpolated_partition_field_end = InterpolatedString.create( + datetime_based_cursor_model.partition_field_end or "end_time", + parameters=datetime_based_cursor_model.parameters or {}, + ) + + slice_boundary_fields = ( + interpolated_partition_field_start.eval(config=config), + interpolated_partition_field_end.eval(config=config), + ) + + datetime_format = datetime_based_cursor_model.datetime_format + + cursor_granularity = ( + parse_duration(datetime_based_cursor_model.cursor_granularity) + if datetime_based_cursor_model.cursor_granularity + else None + ) + + lookback_window = None + interpolated_lookback_window = ( + InterpolatedString.create( + datetime_based_cursor_model.lookback_window, + parameters=datetime_based_cursor_model.parameters or {}, + ) + if datetime_based_cursor_model.lookback_window + else None + ) + if interpolated_lookback_window: + evaluated_lookback_window = interpolated_lookback_window.eval(config=config) + if evaluated_lookback_window: + lookback_window = parse_duration(evaluated_lookback_window) + + connector_state_converter: DateTimeStreamStateConverter + connector_state_converter = CustomFormatConcurrentStreamStateConverter( + datetime_format=datetime_format, + input_datetime_formats=datetime_based_cursor_model.cursor_datetime_formats, + is_sequential_state=True, + cursor_granularity=cursor_granularity, + # type: ignore # Having issues w/ inspection for GapType and CursorValueType as shown in existing tests. Confirmed functionality is working in practice + ) + + start_date_runtime_value: Union[InterpolatedString, str, MinMaxDatetime] + if isinstance(datetime_based_cursor_model.start_datetime, MinMaxDatetimeModel): + start_date_runtime_value = self.create_min_max_datetime( + model=datetime_based_cursor_model.start_datetime, config=config + ) + else: + start_date_runtime_value = datetime_based_cursor_model.start_datetime + + end_date_runtime_value: Optional[Union[InterpolatedString, str, MinMaxDatetime]] + if isinstance(datetime_based_cursor_model.end_datetime, MinMaxDatetimeModel): + end_date_runtime_value = self.create_min_max_datetime( + model=datetime_based_cursor_model.end_datetime, config=config + ) + else: + end_date_runtime_value = datetime_based_cursor_model.end_datetime + + interpolated_start_date = MinMaxDatetime.create( + interpolated_string_or_min_max_datetime=start_date_runtime_value, + parameters=datetime_based_cursor_model.parameters, + ) + interpolated_end_date = ( + None + if not end_date_runtime_value + else MinMaxDatetime.create( + end_date_runtime_value, datetime_based_cursor_model.parameters + ) + ) + + # If datetime format is not specified then start/end datetime should inherit it from the stream slicer + if not interpolated_start_date.datetime_format: + interpolated_start_date.datetime_format = datetime_format + if interpolated_end_date and not interpolated_end_date.datetime_format: + interpolated_end_date.datetime_format = datetime_format + + start_date = interpolated_start_date.get_datetime(config=config) + end_date_provider = ( + partial(interpolated_end_date.get_datetime, config) + if interpolated_end_date + else connector_state_converter.get_end_provider() + ) + + if ( + datetime_based_cursor_model.step and not datetime_based_cursor_model.cursor_granularity + ) or ( + not datetime_based_cursor_model.step and datetime_based_cursor_model.cursor_granularity + ): + raise ValueError( + f"If step is defined, cursor_granularity should be as well and vice-versa. " + f"Right now, step is `{datetime_based_cursor_model.step}` and cursor_granularity is `{datetime_based_cursor_model.cursor_granularity}`" + ) + + # When step is not defined, default to a step size from the starting date to the present moment + step_length = datetime.timedelta.max + interpolated_step = ( + InterpolatedString.create( + datetime_based_cursor_model.step, + parameters=datetime_based_cursor_model.parameters or {}, + ) + if datetime_based_cursor_model.step + else None + ) + if interpolated_step: + evaluated_step = interpolated_step.eval(config) + if evaluated_step: + step_length = parse_duration(evaluated_step) + + cursor_factory = ConcurrentCursorFactory( + partial(self.create_concurrent_cursor_from_datetime_based_cursor, + state_manager = state_manager, + model_type = model_type, + component_definition = component_definition, + stream_name = stream_name, + stream_namespace = stream_namespace, + config = config) + ) + + partition_router = partition_router + + return ( + ConcurrentPerPartitionCursor( + cursor_factory=cursor_factory, + partition_router=partition_router, + stream_name=stream_name, + stream_namespace=stream_namespace, + stream_state=stream_state, + message_repository=self._message_repository, # type: ignore # message_repository is always instantiated with a value by factory + connector_state_manager=state_manager, + connector_state_converter=connector_state_converter, + cursor_field=cursor_field, + slice_boundary_fields=slice_boundary_fields, + start=start_date, # type: ignore # Having issues w/ inspection for GapType and CursorValueType as shown in existing tests. Confirmed functionality is working in practice + end_provider=end_date_provider, # type: ignore # Having issues w/ inspection for GapType and CursorValueType as shown in existing tests. Confirmed functionality is working in practice + lookback_window=lookback_window, + slice_range=step_length, + cursor_granularity=cursor_granularity, + ), + connector_state_converter, + ) @staticmethod def create_constant_backoff_strategy( 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 1c2ad06cf..fd82b0046 100644 --- a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py +++ b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py @@ -39,6 +39,32 @@ def create(self, stream_slice: StreamSlice) -> Partition: stream_slice, ) +class DeclarativePartitionFactory1: + def __init__( + self, + stream_name: str, + json_schema: Mapping[str, Any], + retriever: Retriever, + message_repository: MessageRepository, + ) -> None: + """ + The DeclarativePartitionFactory takes a retriever_factory and not a retriever directly. The reason is that our components are not + thread safe and classes like `DefaultPaginator` may not work because multiple threads can access and modify a shared field across each other. + In order to avoid these problems, we will create one retriever per thread which should make the processing thread-safe. + """ + self._stream_name = stream_name + self._json_schema = json_schema + self._retriever = retriever + self._message_repository = message_repository + + def create(self, stream_slice: StreamSlice) -> Partition: + return DeclarativePartition( + self._stream_name, + self._json_schema, + self._retriever, + self._message_repository, + stream_slice, + ) class DeclarativePartition(Partition): def __init__( diff --git a/airbyte_cdk/sources/streams/concurrent/cursor.py b/airbyte_cdk/sources/streams/concurrent/cursor.py index 1cc7e8965..8a677f81f 100644 --- a/airbyte_cdk/sources/streams/concurrent/cursor.py +++ b/airbyte_cdk/sources/streams/concurrent/cursor.py @@ -220,6 +220,15 @@ def observe(self, record: Record) -> None: def _extract_cursor_value(self, record: Record) -> Any: return self._connector_state_converter.parse_value(self._cursor_field.extract_value(record)) + def close_partition_without_emit(self, partition: Partition) -> None: + slice_count_before = len(self.state.get("slices", [])) + self._add_slice_to_state(partition) + if slice_count_before < len( + self.state["slices"] + ): # only emit if at least one slice has been processed + self._merge_partitions() + self._has_closed_at_least_one_slice = True + def close_partition(self, partition: Partition) -> None: slice_count_before = len(self.state.get("slices", [])) self._add_slice_to_state(partition) From a3304b9a6fb16ac56144e0998d32f12c4cbf7e1c Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 4 Dec 2024 13:34:12 +0200 Subject: [PATCH 02/34] Use request options provider for ConcurrentPerPartitionCursor --- .../concurrent_declarative_source.py | 15 +- .../concurrent_partition_cursor.py | 67 ++------ .../incremental/per_partition_cursor.py | 9 - .../parsers/model_to_component_factory.py | 154 ++---------------- .../retrievers/simple_retriever.py | 2 +- 5 files changed, 40 insertions(+), 207 deletions(-) diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 2b2b957cf..3161d981b 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -3,8 +3,6 @@ # import logging -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import PerPartitionCursor -from airbyte_cdk.sources.declarative.incremental.per_partition_with_global import PerPartitionWithGlobalCursor from typing import Any, Callable, Generic, Iterator, List, Mapping, Optional, Tuple, Union from airbyte_cdk.models import ( @@ -22,6 +20,9 @@ ClientSideIncrementalRecordFilterDecorator, ) from airbyte_cdk.sources.declarative.incremental.datetime_based_cursor import DatetimeBasedCursor +from airbyte_cdk.sources.declarative.incremental.per_partition_with_global import ( + PerPartitionWithGlobalCursor, +) from airbyte_cdk.sources.declarative.interpolation import InterpolatedString from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( @@ -306,8 +307,8 @@ def _group_streams( ) ) elif ( - datetime_based_cursor_component_definition - and datetime_based_cursor_component_definition.get("type", "") + incremental_sync_component_definition + and incremental_sync_component_definition.get("type", "") == DatetimeBasedCursorModel.__name__ and self._stream_supports_concurrent_partition_processing( declarative_stream=declarative_stream @@ -320,18 +321,16 @@ def _group_streams( ) partition_router = declarative_stream.retriever.stream_slicer._partition_router - cursor, connector_state_converter = ( - self._constructor.create_concurrent_cursor_from_perpartition_cursor( + cursor = self._constructor.create_concurrent_cursor_from_perpartition_cursor( state_manager=state_manager, model_type=DatetimeBasedCursorModel, - component_definition=datetime_based_cursor_component_definition, + component_definition=incremental_sync_component_definition, stream_name=declarative_stream.name, stream_namespace=declarative_stream.namespace, config=config or {}, stream_state=stream_state, partition_router=partition_router, ) - ) partition_generator = StreamSlicerPartitionGenerator( diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 74e8ff5f6..6e69f6fae 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -1,46 +1,31 @@ import copy -from airbyte_cdk.sources.streams.concurrent.cursor import Cursor - # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # - import logging from collections import OrderedDict -from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional, Union +from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter +from airbyte_cdk.sources.message import MessageRepository from airbyte_cdk.sources.streams.checkpoint.per_partition_key_serializer import ( PerPartitionKeySerializer, ) -from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, CursorField, CursorValueType, GapType +from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, CursorField from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.types import Record, StreamSlice, StreamState -import functools -from abc import ABC, abstractmethod -from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional, Protocol, Tuple - -from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager -from airbyte_cdk.sources.message import MessageRepository -from airbyte_cdk.sources.streams import NO_CURSOR_STATE_KEY -from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition -from airbyte_cdk.sources.streams.concurrent.partitions.record import Record -from airbyte_cdk.sources.streams.concurrent.partitions.stream_slicer import StreamSlicer -from airbyte_cdk.sources.streams.concurrent.state_converters.abstract_stream_state_converter import ( - AbstractStreamStateConverter, -) -from airbyte_cdk.sources.types import StreamSlice logger = logging.getLogger("airbyte") class ConcurrentCursorFactory: - def __init__(self, create_function: Callable[[Mapping[str, Any]], DeclarativeCursor]): + def __init__(self, create_function: Callable[..., tuple[Cursor, ...]]): self._create_function = create_function - def create(self, stream_state: Mapping[str, Any]) -> DeclarativeCursor: + def create(self, stream_state: Mapping[str, Any]) -> Cursor: return self._create_function(stream_state=stream_state)[0] @@ -78,44 +63,29 @@ def __init__( stream_state: Any, message_repository: MessageRepository, connector_state_manager: ConnectorStateManager, - connector_state_converter: AbstractStreamStateConverter, cursor_field: CursorField, - slice_boundary_fields: Optional[Tuple[str, str]], - start: Optional[CursorValueType], - end_provider: Callable[[], CursorValueType], - lookback_window: Optional[GapType] = None, - slice_range: Optional[GapType] = None, - cursor_granularity: Optional[GapType] = None, ) -> None: self._stream_name = stream_name self._stream_namespace = stream_namespace self._message_repository = message_repository - self._connector_state_converter = connector_state_converter self._connector_state_manager = connector_state_manager self._cursor_field = cursor_field - # To see some example where the slice boundaries might not be defined, check https://github.com/airbytehq/airbyte/blob/1ce84d6396e446e1ac2377362446e3fb94509461/airbyte-integrations/connectors/source-stripe/source_stripe/streams.py#L363-L379 - self._slice_boundary_fields = slice_boundary_fields - self._start = start - self._end_provider = end_provider - self._conccurent_state = stream_state - # self.start, self._concurrent_state = self._get_concurrent_state(stream_state) - self._lookback_window = lookback_window - self._slice_range = slice_range - self._most_recent_cursor_value_per_partition: MutableMapping[Partition, Any] = {} - self._has_closed_at_least_one_slice = False - self._cursor_granularity = cursor_granularity self._cursor_factory = cursor_factory self._partition_router = partition_router + # The dict is ordered to ensure that once the maximum number of partitions is reached, # the oldest partitions can be efficiently removed, maintaining the most recent partitions. self._cursor_per_partition: OrderedDict[str, Cursor] = OrderedDict() self._over_limit = 0 - self._state = {} self._partition_serializer = PerPartitionKeySerializer() self._set_initial_state(stream_state) + @property + def cursor_field(self) -> CursorField: + return self._cursor_field + @property def state(self) -> MutableMapping[str, Any]: states = [] @@ -123,7 +93,6 @@ def state(self) -> MutableMapping[str, Any]: cursor_state = cursor._connector_state_converter.convert_to_state_message( cursor._cursor_field, cursor.state ) - # print(cursor_state, cursor.state) if cursor_state: states.append( { @@ -132,13 +101,8 @@ def state(self) -> MutableMapping[str, Any]: } ) state: dict[str, Any] = {"states": states} - # print(state) return state - @property - def cursor_field(self) -> CursorField: - return self._cursor_field - def close_partition(self, partition: Partition) -> None: self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)].close_partition_without_emit(partition=partition) @@ -253,8 +217,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._partition_router.set_initial_state(stream_state) def observe(self, record: Record) -> None: - print(f"ESTATE: {self._to_partition_key(record.partition._stream_slice.partition)}: {record.data[self.cursor_field.cursor_field_key]}") - self._cursor_per_partition[self._to_partition_key(record.partition._stream_slice.partition)].observe(record) + self._cursor_per_partition[self._to_partition_key(record.associated_slice.partition)].observe(record) def _to_partition_key(self, partition: Mapping[str, Any]) -> str: return self._partition_serializer.to_partition_key(partition) @@ -267,9 +230,7 @@ def _create_cursor(self, cursor_state: Any) -> DeclarativeCursor: return cursor def should_be_synced(self, record: Record) -> bool: - return self._get_cursor(record).should_be_synced( - self._convert_record_to_cursor_record(record) - ) + return self._get_cursor(record).should_be_synced(record) def is_greater_than_or_equal(self, first: Record, second: Record) -> bool: if not first.associated_slice or not second.associated_slice: @@ -295,7 +256,7 @@ def _convert_record_to_cursor_record(record: Record) -> Record: else None, ) - def _get_cursor(self, record: Record) -> DeclarativeCursor: + def _get_cursor(self, record: Record) -> Cursor: if not record.associated_slice: raise ValueError( "Invalid state as stream slices that are emitted should refer to an existing cursor" diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py index 99765319f..8241f7761 100644 --- a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py @@ -244,15 +244,6 @@ def get_request_headers( next_page_token: Optional[Mapping[str, Any]] = None, ) -> Mapping[str, Any]: if stream_slice: - if self._to_partition_key(stream_slice.partition) not in self._cursor_per_partition: - partition_state = ( - self._state_to_migrate_from - if self._state_to_migrate_from - else self._NO_CURSOR_STATE - ) - cursor = self._create_cursor(partition_state) - - self._cursor_per_partition[self._to_partition_key(stream_slice.partition)] = cursor return self._partition_router.get_request_headers( # type: ignore # this always returns a mapping stream_state=stream_state, stream_slice=StreamSlice(partition=stream_slice.partition, cursor_slice={}), 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 0ba9dc632..8d9d4a9e3 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -82,8 +82,8 @@ ) from airbyte_cdk.sources.declarative.incremental import ( ChildPartitionResumableFullRefreshCursor, - ConcurrentPerPartitionCursor, ConcurrentCursorFactory, + ConcurrentPerPartitionCursor, CursorFactory, DatetimeBasedCursor, DeclarativeCursor, @@ -911,6 +911,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( ), connector_state_converter, ) + def create_concurrent_cursor_from_perpartition_cursor( self, state_manager: ConnectorStateManager, @@ -922,7 +923,7 @@ def create_concurrent_cursor_from_perpartition_cursor( stream_state: MutableMapping[str, Any], partition_router, **kwargs: Any, - ) -> Tuple[ConcurrentCursor, DateTimeStreamStateConverter]: + ) -> ConcurrentPerPartitionCursor: component_type = component_definition.get("type") if component_definition.get("type") != model_type.__name__: raise ValueError( @@ -942,149 +943,30 @@ def create_concurrent_cursor_from_perpartition_cursor( ) cursor_field = CursorField(interpolated_cursor_field.eval(config=config)) - interpolated_partition_field_start = InterpolatedString.create( - datetime_based_cursor_model.partition_field_start or "start_time", - parameters=datetime_based_cursor_model.parameters or {}, - ) - interpolated_partition_field_end = InterpolatedString.create( - datetime_based_cursor_model.partition_field_end or "end_time", - parameters=datetime_based_cursor_model.parameters or {}, - ) - - slice_boundary_fields = ( - interpolated_partition_field_start.eval(config=config), - interpolated_partition_field_end.eval(config=config), - ) - - datetime_format = datetime_based_cursor_model.datetime_format - - cursor_granularity = ( - parse_duration(datetime_based_cursor_model.cursor_granularity) - if datetime_based_cursor_model.cursor_granularity - else None - ) - - lookback_window = None - interpolated_lookback_window = ( - InterpolatedString.create( - datetime_based_cursor_model.lookback_window, - parameters=datetime_based_cursor_model.parameters or {}, - ) - if datetime_based_cursor_model.lookback_window - else None - ) - if interpolated_lookback_window: - evaluated_lookback_window = interpolated_lookback_window.eval(config=config) - if evaluated_lookback_window: - lookback_window = parse_duration(evaluated_lookback_window) - - connector_state_converter: DateTimeStreamStateConverter - connector_state_converter = CustomFormatConcurrentStreamStateConverter( - datetime_format=datetime_format, - input_datetime_formats=datetime_based_cursor_model.cursor_datetime_formats, - is_sequential_state=True, - cursor_granularity=cursor_granularity, - # type: ignore # Having issues w/ inspection for GapType and CursorValueType as shown in existing tests. Confirmed functionality is working in practice - ) - - start_date_runtime_value: Union[InterpolatedString, str, MinMaxDatetime] - if isinstance(datetime_based_cursor_model.start_datetime, MinMaxDatetimeModel): - start_date_runtime_value = self.create_min_max_datetime( - model=datetime_based_cursor_model.start_datetime, config=config - ) - else: - start_date_runtime_value = datetime_based_cursor_model.start_datetime - - end_date_runtime_value: Optional[Union[InterpolatedString, str, MinMaxDatetime]] - if isinstance(datetime_based_cursor_model.end_datetime, MinMaxDatetimeModel): - end_date_runtime_value = self.create_min_max_datetime( - model=datetime_based_cursor_model.end_datetime, config=config - ) - else: - end_date_runtime_value = datetime_based_cursor_model.end_datetime - - interpolated_start_date = MinMaxDatetime.create( - interpolated_string_or_min_max_datetime=start_date_runtime_value, - parameters=datetime_based_cursor_model.parameters, - ) - interpolated_end_date = ( - None - if not end_date_runtime_value - else MinMaxDatetime.create( - end_date_runtime_value, datetime_based_cursor_model.parameters - ) - ) - - # If datetime format is not specified then start/end datetime should inherit it from the stream slicer - if not interpolated_start_date.datetime_format: - interpolated_start_date.datetime_format = datetime_format - if interpolated_end_date and not interpolated_end_date.datetime_format: - interpolated_end_date.datetime_format = datetime_format - - start_date = interpolated_start_date.get_datetime(config=config) - end_date_provider = ( - partial(interpolated_end_date.get_datetime, config) - if interpolated_end_date - else connector_state_converter.get_end_provider() - ) - - if ( - datetime_based_cursor_model.step and not datetime_based_cursor_model.cursor_granularity - ) or ( - not datetime_based_cursor_model.step and datetime_based_cursor_model.cursor_granularity - ): - raise ValueError( - f"If step is defined, cursor_granularity should be as well and vice-versa. " - f"Right now, step is `{datetime_based_cursor_model.step}` and cursor_granularity is `{datetime_based_cursor_model.cursor_granularity}`" - ) - - # When step is not defined, default to a step size from the starting date to the present moment - step_length = datetime.timedelta.max - interpolated_step = ( - InterpolatedString.create( - datetime_based_cursor_model.step, - parameters=datetime_based_cursor_model.parameters or {}, - ) - if datetime_based_cursor_model.step - else None - ) - if interpolated_step: - evaluated_step = interpolated_step.eval(config) - if evaluated_step: - step_length = parse_duration(evaluated_step) - + # Create the cursor factory cursor_factory = ConcurrentCursorFactory( - partial(self.create_concurrent_cursor_from_datetime_based_cursor, - state_manager = state_manager, - model_type = model_type, - component_definition = component_definition, - stream_name = stream_name, - stream_namespace = stream_namespace, - config = config) + partial( + self.create_concurrent_cursor_from_datetime_based_cursor, + state_manager=state_manager, + model_type=model_type, + component_definition=component_definition, + stream_name=stream_name, + stream_namespace=stream_namespace, + config=config, + ) ) - partition_router = partition_router - - return ( - ConcurrentPerPartitionCursor( + # Return the concurrent cursor and state converter + return ConcurrentPerPartitionCursor( cursor_factory=cursor_factory, partition_router=partition_router, stream_name=stream_name, stream_namespace=stream_namespace, stream_state=stream_state, - message_repository=self._message_repository, # type: ignore # message_repository is always instantiated with a value by factory + message_repository=self._message_repository, # type: ignore connector_state_manager=state_manager, - connector_state_converter=connector_state_converter, cursor_field=cursor_field, - slice_boundary_fields=slice_boundary_fields, - start=start_date, # type: ignore # Having issues w/ inspection for GapType and CursorValueType as shown in existing tests. Confirmed functionality is working in practice - end_provider=end_date_provider, # type: ignore # Having issues w/ inspection for GapType and CursorValueType as shown in existing tests. Confirmed functionality is working in practice - lookback_window=lookback_window, - slice_range=step_length, - cursor_granularity=cursor_granularity, - ), - connector_state_converter, - ) + ) @staticmethod def create_constant_backoff_strategy( @@ -2092,7 +1974,7 @@ def create_simple_retriever( if ( not isinstance(stream_slicer, DatetimeBasedCursor) or type(stream_slicer) is not DatetimeBasedCursor - ): + ) and not isinstance(stream_slicer, PerPartitionWithGlobalCursor): # Many of the custom component implementations of DatetimeBasedCursor override get_request_params() (or other methods). # Because we're decoupling RequestOptionsProvider from the Cursor, custom components will eventually need to reimplement # their own RequestOptionsProvider. However, right now the existing StreamSlicer/Cursor still can act as the SimpleRetriever's diff --git a/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py b/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py index cc7040595..3936dda4d 100644 --- a/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py +++ b/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py @@ -178,7 +178,7 @@ def _request_headers( stream_slice, next_page_token, self._paginator.get_request_headers, - self.stream_slicer.get_request_headers, + self.request_option_provider.get_request_headers, ) if isinstance(headers, str): raise ValueError("Request headers cannot be a string") From 4ddbb84d5ffdb4ab4cc462a678379c96674781bd Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 4 Dec 2024 13:37:44 +0200 Subject: [PATCH 03/34] Delete unused DeclarativePartitionFactory --- .../declarative_partition_generator.py | 27 ------------------- 1 file changed, 27 deletions(-) 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 c7e60ba4d..09ed2bc80 100644 --- a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py +++ b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py @@ -38,33 +38,6 @@ def create(self, stream_slice: StreamSlice) -> Partition: stream_slice, ) -class DeclarativePartitionFactory1: - def __init__( - self, - stream_name: str, - json_schema: Mapping[str, Any], - retriever: Retriever, - message_repository: MessageRepository, - ) -> None: - """ - The DeclarativePartitionFactory takes a retriever_factory and not a retriever directly. The reason is that our components are not - thread safe and classes like `DefaultPaginator` may not work because multiple threads can access and modify a shared field across each other. - In order to avoid these problems, we will create one retriever per thread which should make the processing thread-safe. - """ - self._stream_name = stream_name - self._json_schema = json_schema - self._retriever = retriever - self._message_repository = message_repository - - def create(self, stream_slice: StreamSlice) -> Partition: - return DeclarativePartition( - self._stream_name, - self._json_schema, - self._retriever, - self._message_repository, - stream_slice, - ) - class DeclarativePartition(Partition): def __init__( self, From 41b029d33321f6eacf8482c4b725ae2c32587714 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 5 Dec 2024 15:17:55 +0200 Subject: [PATCH 04/34] Fixed record filter --- .../declarative/extractors/record_filter.py | 8 +++----- .../incremental/per_partition_cursor.py | 9 +++++++++ .../parsers/model_to_component_factory.py | 17 +++++++---------- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/airbyte_cdk/sources/declarative/extractors/record_filter.py b/airbyte_cdk/sources/declarative/extractors/record_filter.py index b744c9796..373669612 100644 --- a/airbyte_cdk/sources/declarative/extractors/record_filter.py +++ b/airbyte_cdk/sources/declarative/extractors/record_filter.py @@ -59,13 +59,11 @@ class ClientSideIncrementalRecordFilterDecorator(RecordFilter): def __init__( self, - date_time_based_cursor: DatetimeBasedCursor, - substream_cursor: Optional[Union[PerPartitionWithGlobalCursor, GlobalSubstreamCursor]], + cursor: Union[DatetimeBasedCursor, PerPartitionWithGlobalCursor, GlobalSubstreamCursor], **kwargs: Any, ): super().__init__(**kwargs) - self._date_time_based_cursor = date_time_based_cursor - self._substream_cursor = substream_cursor + self._cursor = cursor def filter_records( self, @@ -77,7 +75,7 @@ def filter_records( records = ( record for record in records - if (self._substream_cursor or self._date_time_based_cursor).should_be_synced( + if self._cursor.should_be_synced( # Record is created on the fly to align with cursors interface; stream name is ignored as we don't need it here # Record stream name is empty cause it is not used durig the filtering Record(data=record, associated_slice=stream_slice, stream_name="") diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py index 8241f7761..d7322709d 100644 --- a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py @@ -303,6 +303,15 @@ def get_request_body_json( raise ValueError("A partition needs to be provided in order to get request body json") def should_be_synced(self, record: Record) -> bool: + if self._to_partition_key(record.associated_slice.partition) not in self._cursor_per_partition: + partition_state = ( + self._state_to_migrate_from + if self._state_to_migrate_from + else self._NO_CURSOR_STATE + ) + cursor = self._create_cursor(partition_state) + + self._cursor_per_partition[self._to_partition_key(record.associated_slice.partition)] = cursor return self._get_cursor(record).should_be_synced( self._convert_record_to_cursor_record(record) ) 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 8d9d4a9e3..206870e21 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -1250,17 +1250,14 @@ def create_declarative_stream( raise ValueError( "Unsupported Slicer is used. PerPartitionWithGlobalCursor should be used here instead" ) + cursor = combined_slicers if isinstance( + combined_slicers, (PerPartitionWithGlobalCursor, GlobalSubstreamCursor) + ) else self._create_component_from_model( + model=model.incremental_sync, config=config + ) + client_side_incremental_sync = { - "date_time_based_cursor": self._create_component_from_model( - model=model.incremental_sync, config=config - ), - "substream_cursor": ( - combined_slicers - if isinstance( - combined_slicers, (PerPartitionWithGlobalCursor, GlobalSubstreamCursor) - ) - else None - ), + "cursor": cursor } if model.incremental_sync and isinstance(model.incremental_sync, DatetimeBasedCursorModel): From dfcf17fc858f0f58619b0f17e2509bfdae16ef87 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 5 Dec 2024 16:13:04 +0200 Subject: [PATCH 05/34] Add unit test --- .../concurrent_partition_cursor.py | 4 +- .../test_concurrent_perpartitioncursor.py | 523 ++++++++++++++++++ 2 files changed, 525 insertions(+), 2 deletions(-) create mode 100644 unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 6e69f6fae..dc89e5ddb 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -22,11 +22,11 @@ class ConcurrentCursorFactory: - def __init__(self, create_function: Callable[..., tuple[Cursor, ...]]): + def __init__(self, create_function: Callable[..., Cursor]): self._create_function = create_function def create(self, stream_state: Mapping[str, Any]) -> Cursor: - return self._create_function(stream_state=stream_state)[0] + return self._create_function(stream_state=stream_state) class ConcurrentPerPartitionCursor(Cursor): diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py new file mode 100644 index 000000000..bbbcfdc2a --- /dev/null +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -0,0 +1,523 @@ +# Copyright (c) 2024 Airbyte, Inc., all rights reserved. + +from typing import Any, List, Mapping, MutableMapping, Optional, Union +from unittest.mock import MagicMock + +import pytest +import requests_mock +from orjson import orjson + +from airbyte_cdk.models import ( + AirbyteMessage, + AirbyteStateBlob, + AirbyteStateMessage, + AirbyteStateType, + AirbyteStream, + AirbyteStreamState, + ConfiguredAirbyteCatalog, + ConfiguredAirbyteStream, + DestinationSyncMode, + StreamDescriptor, + SyncMode, +) +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) + +SUBSTREAM_MANIFEST: MutableMapping[str, Any] = { + "version": "0.51.42", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["post_comment_votes"]}, + "definitions": { + "basic_authenticator": { + "type": "BasicHttpAuthenticator", + "username": "{{ config['credentials']['email'] + '/token' }}", + "password": "{{ config['credentials']['api_token'] }}", + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.example.com", + "http_method": "GET", + "authenticator": "#/definitions/basic_authenticator", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": { + "type": "DpathExtractor", + "field_path": ["{{ parameters.get('data_path') or parameters['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": "{{ not response.get('next_page', {}) }}", + }, + "page_token_option": {"type": "RequestPath"}, + }, + }, + "cursor_incremental_sync": { + "type": "DatetimeBasedCursor", + "cursor_datetime_formats": ["%Y-%m-%dT%H:%M:%SZ", "%Y-%m-%dT%H:%M:%S%z"], + "datetime_format": "%Y-%m-%dT%H:%M:%SZ", + "cursor_field": "{{ parameters.get('cursor_field', 'updated_at') }}", + "start_datetime": {"datetime": "{{ config.get('start_date')}}"}, + "start_time_option": { + "inject_into": "request_parameter", + "field_name": "start_time", + "type": "RequestOption", + }, + }, + "posts_stream": { + "type": "DeclarativeStream", + "name": "posts", + "primary_key": ["id"], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "id": {"type": "integer"}, + "updated_at": {"type": "string", "format": "date-time"}, + "title": {"type": "string"}, + "content": {"type": "string"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.example.com", + "path": "/community/posts", + "http_method": "GET", + "authenticator": "#/definitions/basic_authenticator", + }, + "record_selector": "#/definitions/retriever/record_selector", + "paginator": "#/definitions/retriever/paginator", + }, + "incremental_sync": "#/definitions/cursor_incremental_sync", + "$parameters": { + "name": "posts", + "path": "community/posts", + "data_path": "posts", + "cursor_field": "updated_at", + "primary_key": "id", + }, + }, + "post_comments_stream": { + "type": "DeclarativeStream", + "name": "post_comments", + "primary_key": ["id"], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "id": {"type": "integer"}, + "updated_at": {"type": "string", "format": "date-time"}, + "post_id": {"type": "integer"}, + "comment": {"type": "string"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.example.com", + "path": "/community/posts/{{ stream_slice.id }}/comments", + "http_method": "GET", + "authenticator": "#/definitions/basic_authenticator", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["comments"]}, + "record_filter": { + "condition": "{{ record['updated_at'] >= stream_state.get('updated_at', config.get('start_date')) }}" + }, + }, + "paginator": "#/definitions/retriever/paginator", + "partition_router": { + "type": "SubstreamPartitionRouter", + "parent_stream_configs": [ + { + "stream": "#/definitions/posts_stream", + "parent_key": "id", + "partition_field": "id", + "incremental_dependency": True, + } + ], + }, + }, + "incremental_sync": { + "type": "DatetimeBasedCursor", + "cursor_datetime_formats": ["%Y-%m-%dT%H:%M:%SZ", "%Y-%m-%dT%H:%M:%S%z"], + "datetime_format": "%Y-%m-%dT%H:%M:%SZ", + "cursor_field": "{{ parameters.get('cursor_field', 'updated_at') }}", + "start_datetime": {"datetime": "{{ config.get('start_date') }}"}, + }, + "$parameters": { + "name": "post_comments", + "path": "community/posts/{{ stream_slice.id }}/comments", + "data_path": "comments", + "cursor_field": "updated_at", + "primary_key": "id", + }, + }, + "post_comment_votes_stream": { + "type": "DeclarativeStream", + "name": "post_comment_votes", + "primary_key": ["id"], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "id": {"type": "integer"}, + "created_at": {"type": "string", "format": "date-time"}, + "comment_id": {"type": "integer"}, + "vote": {"type": "number"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.example.com", + "path": "/community/posts/{{ stream_slice.parent_slice.id }}/comments/{{ stream_slice.id }}/votes", + "http_method": "GET", + "authenticator": "#/definitions/basic_authenticator", + }, + "record_selector": "#/definitions/retriever/record_selector", + "paginator": "#/definitions/retriever/paginator", + "partition_router": { + "type": "SubstreamPartitionRouter", + "parent_stream_configs": [ + { + "stream": "#/definitions/post_comments_stream", + "parent_key": "id", + "partition_field": "id", + "incremental_dependency": True, + } + ], + }, + }, + "incremental_sync": "#/definitions/cursor_incremental_sync", + "$parameters": { + "name": "post_comment_votes", + "path": "community/posts/{{ stream_slice.parent_slice.id }}/comments/{{ stream_slice.id }}/votes", + "data_path": "votes", + "cursor_field": "created_at", + "primary_key": "id", + }, + }, + }, + "streams": [ + {"$ref": "#/definitions/posts_stream"}, + {"$ref": "#/definitions/post_comments_stream"}, + {"$ref": "#/definitions/post_comment_votes_stream"}, + ], + "concurrency_level": { + "type": "ConcurrencyLevel", + "default_concurrency": "{{ config['num_workers'] or 10 }}", + "max_concurrency": 25, + }, +} + + +def _run_read( + manifest: Mapping[str, Any], + config: Mapping[str, Any], + stream_name: str, + state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, +) -> List[AirbyteMessage]: + catalog = ConfiguredAirbyteCatalog( + streams=[ + ConfiguredAirbyteStream( + stream=AirbyteStream( + name=stream_name, + json_schema={}, + supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], + ), + sync_mode=SyncMode.incremental, + destination_sync_mode=DestinationSyncMode.append, + ) + ] + ) + logger = MagicMock() + source = ConcurrentDeclarativeSource( + source_config=manifest, config=config, catalog=catalog, state=state + ) + messages = list( + source.read(logger=source.logger, config=config, catalog=catalog, state=[]) + ) + return messages + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + SUBSTREAM_MANIFEST, + [ + # Fetch the first page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "posts": [ + {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, + {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + }, + ), + # Fetch the second page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-01T00:00:01Z&page=2", + {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + }, + ), + # Fetch the second page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + { + "votes": [ + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} + ] + }, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 30 of post 3 + ( + "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + { + "votes": [ + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} + ] + }, + ), + ], + # Expected records + [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"}, + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + ], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob( + { + # This should not happen since parent state is disabled, but I've added this to validate that and + # incoming parent_state is ignored when the parent stream's incremental_dependency is disabled + "parent_state": { + "post_comments": { + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + } + ], + "parent_state": { + "posts": {"updated_at": "2024-01-05T00:00:00Z"} + }, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + ], + } + ), + ), + ) + ], + # Expected state + { + "states": [ + { + "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + }, + { + "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + }, + { + "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + }, + { + "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:00Z"}, + }, + { + "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + }, + { + "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + }, + ], + }, + ), + ], +) +def test_incremental_parent_state_no_incremental_dependency( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + """ + This is a pretty complicated test that syncs a low-code connector stream with three levels of substreams + - posts: (ids: 1, 2, 3) + - post comments: (parent post 1 with ids: 9, 10, 11, 12; parent post 2 with ids: 20, 21; parent post 3 with id: 30) + - post comment votes: (parent comment 10 with ids: 100, 101; parent comment 11 with id: 102; + parent comment 20 with id: 200; parent comment 21 with id: 201, parent comment 30 with id: 300) + + By setting incremental_dependency to false, parent streams will not use the incoming state and will not update state. + The post_comment_votes substream is incremental and will emit state messages We verify this by ensuring that mocked + parent stream requests use the incoming config as query parameters and the substream state messages does not + contain parent stream state. + """ + + _stream_name = "post_comment_votes" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + # Disable incremental_dependency + manifest["definitions"]["post_comments_stream"]["retriever"]["partition_router"][ + "parent_stream_configs" + ][0]["incremental_dependency"] = False + manifest["definitions"]["post_comment_votes_stream"]["retriever"]["partition_router"][ + "parent_stream_configs" + ][0]["incremental_dependency"] = False + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert set(tuple(sorted(d.items())) for d in output_data) == set(tuple(sorted(d.items())) for d in expected_records) + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state From b84e68a3953324165eb0128a4dbb473ff99da114 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Fri, 6 Dec 2024 15:48:54 +0200 Subject: [PATCH 06/34] Fix record filter unit tests --- .../sources/declarative/extractors/test_record_filter.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/unit_tests/sources/declarative/extractors/test_record_filter.py b/unit_tests/sources/declarative/extractors/test_record_filter.py index 12f06a94e..5df391327 100644 --- a/unit_tests/sources/declarative/extractors/test_record_filter.py +++ b/unit_tests/sources/declarative/extractors/test_record_filter.py @@ -290,8 +290,7 @@ def test_client_side_record_filter_decorator_no_parent_stream( config={}, condition=record_filter_expression, parameters={}, - date_time_based_cursor=date_time_based_cursor, - substream_cursor=None, + cursor=date_time_based_cursor, ) filtered_records = list( @@ -429,8 +428,7 @@ def date_time_based_cursor_factory() -> DatetimeBasedCursor: record_filter_decorator = ClientSideIncrementalRecordFilterDecorator( config={}, parameters={}, - date_time_based_cursor=date_time_based_cursor, - substream_cursor=substream_cursor, + cursor=substream_cursor or date_time_based_cursor, ) # The partition we're testing From 2038075796cc71648c94d277f1c11b4f33c36aff Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Fri, 6 Dec 2024 15:51:43 +0200 Subject: [PATCH 07/34] Update poetry lock --- poetry.lock | 587 +++++++++++++++++++++++----------------------------- 1 file changed, 263 insertions(+), 324 deletions(-) diff --git a/poetry.lock b/poetry.lock index b7b9736bf..6ba19ba3d 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.6.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.8.3 and should not be changed by hand. [[package]] name = "aiohappyeyeballs" @@ -13,87 +13,87 @@ files = [ [[package]] name = "aiohttp" -version = "3.11.9" +version = "3.11.10" description = "Async http client/server framework (asyncio)" optional = true python-versions = ">=3.9" files = [ - {file = "aiohttp-3.11.9-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:0411777249f25d11bd2964a230b3ffafcbed6cd65d0f2b132bc2b8f5b8c347c7"}, - {file = "aiohttp-3.11.9-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:499368eb904566fbdf1a3836a1532000ef1308f34a1bcbf36e6351904cced771"}, - {file = "aiohttp-3.11.9-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0b5a5009b0159a8f707879dc102b139466d8ec6db05103ec1520394fdd8ea02c"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:176f8bb8931da0613bb0ed16326d01330066bb1e172dd97e1e02b1c27383277b"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6435a66957cdba1a0b16f368bde03ce9c79c57306b39510da6ae5312a1a5b2c1"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:202f40fb686e5f93908eee0c75d1e6fbe50a43e9bd4909bf3bf4a56b560ca180"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:39625703540feb50b6b7f938b3856d1f4886d2e585d88274e62b1bd273fae09b"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c6beeac698671baa558e82fa160be9761cf0eb25861943f4689ecf9000f8ebd0"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:96726839a42429318017e67a42cca75d4f0d5248a809b3cc2e125445edd7d50d"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:3f5461c77649358610fb9694e790956b4238ac5d9e697a17f63619c096469afe"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:4313f3bc901255b22f01663eeeae167468264fdae0d32c25fc631d5d6e15b502"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:d6e274661c74195708fc4380a4ef64298926c5a50bb10fbae3d01627d7a075b7"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:db2914de2559809fdbcf3e48f41b17a493b58cb7988d3e211f6b63126c55fe82"}, - {file = "aiohttp-3.11.9-cp310-cp310-win32.whl", hash = "sha256:27935716f8d62c1c73010428db310fd10136002cfc6d52b0ba7bdfa752d26066"}, - {file = "aiohttp-3.11.9-cp310-cp310-win_amd64.whl", hash = "sha256:afbe85b50ade42ddff5669947afde9e8a610e64d2c80be046d67ec4368e555fa"}, - {file = "aiohttp-3.11.9-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:afcda759a69c6a8be3aae764ec6733155aa4a5ad9aad4f398b52ba4037942fe3"}, - {file = "aiohttp-3.11.9-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c5bba6b83fde4ca233cfda04cbd4685ab88696b0c8eaf76f7148969eab5e248a"}, - {file = "aiohttp-3.11.9-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:442356e8924fe1a121f8c87866b0ecdc785757fd28924b17c20493961b3d6697"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f737fef6e117856400afee4f17774cdea392b28ecf058833f5eca368a18cf1bf"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ea142255d4901b03f89cb6a94411ecec117786a76fc9ab043af8f51dd50b5313"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6e1e9e447856e9b7b3d38e1316ae9a8c92e7536ef48373de758ea055edfd5db5"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e7f6173302f8a329ca5d1ee592af9e628d3ade87816e9958dcf7cdae2841def7"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a7c6147c6306f537cff59409609508a1d2eff81199f0302dd456bb9e7ea50c39"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:e9d036a9a41fc78e8a3f10a86c2fc1098fca8fab8715ba9eb999ce4788d35df0"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:2ac9fd83096df36728da8e2f4488ac3b5602238f602706606f3702f07a13a409"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:d3108f0ad5c6b6d78eec5273219a5bbd884b4aacec17883ceefaac988850ce6e"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:96bbec47beb131bbf4bae05d8ef99ad9e5738f12717cfbbf16648b78b0232e87"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:fc726c3fa8f606d07bd2b500e5dc4c0fd664c59be7788a16b9e34352c50b6b6b"}, - {file = "aiohttp-3.11.9-cp311-cp311-win32.whl", hash = "sha256:5720ebbc7a1b46c33a42d489d25d36c64c419f52159485e55589fbec648ea49a"}, - {file = "aiohttp-3.11.9-cp311-cp311-win_amd64.whl", hash = "sha256:17af09d963fa1acd7e4c280e9354aeafd9e3d47eaa4a6bfbd2171ad7da49f0c5"}, - {file = "aiohttp-3.11.9-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:c1f2d7fd583fc79c240094b3e7237d88493814d4b300d013a42726c35a734bc9"}, - {file = "aiohttp-3.11.9-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:d4b8a1b6c7a68c73191f2ebd3bf66f7ce02f9c374e309bdb68ba886bbbf1b938"}, - {file = "aiohttp-3.11.9-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:bd3f711f4c99da0091ced41dccdc1bcf8be0281dc314d6d9c6b6cf5df66f37a9"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:44cb1a1326a0264480a789e6100dc3e07122eb8cd1ad6b784a3d47d13ed1d89c"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7a7ddf981a0b953ade1c2379052d47ccda2f58ab678fca0671c7c7ca2f67aac2"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6ffa45cc55b18d4ac1396d1ddb029f139b1d3480f1594130e62bceadf2e1a838"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cca505829cdab58c2495ff418c96092d225a1bbd486f79017f6de915580d3c44"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:44d323aa80a867cb6db6bebb4bbec677c6478e38128847f2c6b0f70eae984d72"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:b2fab23003c4bb2249729a7290a76c1dda38c438300fdf97d4e42bf78b19c810"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:be0c7c98e38a1e3ad7a6ff64af8b6d6db34bf5a41b1478e24c3c74d9e7f8ed42"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:5cc5e0d069c56645446c45a4b5010d4b33ac6c5ebfd369a791b5f097e46a3c08"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9bcf97b971289be69638d8b1b616f7e557e1342debc7fc86cf89d3f08960e411"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c7333e7239415076d1418dbfb7fa4df48f3a5b00f8fdf854fca549080455bc14"}, - {file = "aiohttp-3.11.9-cp312-cp312-win32.whl", hash = "sha256:9384b07cfd3045b37b05ed002d1c255db02fb96506ad65f0f9b776b762a7572e"}, - {file = "aiohttp-3.11.9-cp312-cp312-win_amd64.whl", hash = "sha256:f5252ba8b43906f206048fa569debf2cd0da0316e8d5b4d25abe53307f573941"}, - {file = "aiohttp-3.11.9-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:282e0a7ddd36ebc411f156aeaa0491e8fe7f030e2a95da532cf0c84b0b70bc66"}, - {file = "aiohttp-3.11.9-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:ebd3e6b0c7d4954cca59d241970011f8d3327633d555051c430bd09ff49dc494"}, - {file = "aiohttp-3.11.9-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:30f9f89ae625d412043f12ca3771b2ccec227cc93b93bb1f994db6e1af40a7d3"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7a3b5b2c012d70c63d9d13c57ed1603709a4d9d7d473e4a9dfece0e4ea3d5f51"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6ef1550bb5f55f71b97a6a395286db07f7f2c01c8890e613556df9a51da91e8d"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:317251b9c9a2f1a9ff9cd093775b34c6861d1d7df9439ce3d32a88c275c995cd"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21cbe97839b009826a61b143d3ca4964c8590d7aed33d6118125e5b71691ca46"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:618b18c3a2360ac940a5503da14fa4f880c5b9bc315ec20a830357bcc62e6bae"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a0cf4d814689e58f57ecd5d8c523e6538417ca2e72ff52c007c64065cef50fb2"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:15c4e489942d987d5dac0ba39e5772dcbed4cc9ae3710d1025d5ba95e4a5349c"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:ec8df0ff5a911c6d21957a9182402aad7bf060eaeffd77c9ea1c16aecab5adbf"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:ed95d66745f53e129e935ad726167d3a6cb18c5d33df3165974d54742c373868"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:647ec5bee7e4ec9f1034ab48173b5fa970d9a991e565549b965e93331f1328fe"}, - {file = "aiohttp-3.11.9-cp313-cp313-win32.whl", hash = "sha256:ef2c9499b7bd1e24e473dc1a85de55d72fd084eea3d8bdeec7ee0720decb54fa"}, - {file = "aiohttp-3.11.9-cp313-cp313-win_amd64.whl", hash = "sha256:84de955314aa5e8d469b00b14d6d714b008087a0222b0f743e7ffac34ef56aff"}, - {file = "aiohttp-3.11.9-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:e738aabff3586091221044b7a584865ddc4d6120346d12e28e788307cd731043"}, - {file = "aiohttp-3.11.9-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:28f29bce89c3b401a53d6fd4bee401ee943083bf2bdc12ef297c1d63155070b0"}, - {file = "aiohttp-3.11.9-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:31de2f10f63f96cc19e04bd2df9549559beadd0b2ee2da24a17e7ed877ca8c60"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77f31cebd8c27a36af6c7346055ac564946e562080ee1a838da724585c67474f"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0bcb7f6976dc0b6b56efde13294862adf68dd48854111b422a336fa729a82ea6"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1a8b13b9950d8b2f8f58b6e5842c4b842b5887e2c32e3f4644d6642f1659a530"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c9c23e62f3545c2216100603614f9e019e41b9403c47dd85b8e7e5015bf1bde0"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ec656680fc53a13f849c71afd0c84a55c536206d524cbc831cde80abbe80489e"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:36df00e0541f264ce42d62280281541a47474dfda500bc5b7f24f70a7f87be7a"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:8dcfd14c712aa9dd18049280bfb2f95700ff6a8bde645e09f17c3ed3f05a0130"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:14624d96f0d69cf451deed3173079a68c322279be6030208b045ab77e1e8d550"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:4b01d9cfcb616eeb6d40f02e66bebfe7b06d9f2ef81641fdd50b8dd981166e0b"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:928f92f80e2e8d6567b87d3316c1fd9860ccfe36e87a9a7f5237d4cda8baa1ba"}, - {file = "aiohttp-3.11.9-cp39-cp39-win32.whl", hash = "sha256:c8a02f74ae419e3955af60f570d83187423e42e672a6433c5e292f1d23619269"}, - {file = "aiohttp-3.11.9-cp39-cp39-win_amd64.whl", hash = "sha256:0a97d657f6cf8782a830bb476c13f7d777cfcab8428ac49dde15c22babceb361"}, - {file = "aiohttp-3.11.9.tar.gz", hash = "sha256:a9266644064779840feec0e34f10a89b3ff1d2d6b751fe90017abcad1864fa7c"}, + {file = "aiohttp-3.11.10-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cbad88a61fa743c5d283ad501b01c153820734118b65aee2bd7dbb735475ce0d"}, + {file = "aiohttp-3.11.10-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:80886dac673ceaef499de2f393fc80bb4481a129e6cb29e624a12e3296cc088f"}, + {file = "aiohttp-3.11.10-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:61b9bae80ed1f338c42f57c16918853dc51775fb5cb61da70d590de14d8b5fb4"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9e2e576caec5c6a6b93f41626c9c02fc87cd91538b81a3670b2e04452a63def6"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:02c13415b5732fb6ee7ff64583a5e6ed1c57aa68f17d2bda79c04888dfdc2769"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4cfce37f31f20800a6a6620ce2cdd6737b82e42e06e6e9bd1b36f546feb3c44f"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3bbbfff4c679c64e6e23cb213f57cc2c9165c9a65d63717108a644eb5a7398df"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:49c7dbbc1a559ae14fc48387a115b7d4bbc84b4a2c3b9299c31696953c2a5219"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:68386d78743e6570f054fe7949d6cb37ef2b672b4d3405ce91fafa996f7d9b4d"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:9ef405356ba989fb57f84cac66f7b0260772836191ccefbb987f414bcd2979d9"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:5d6958671b296febe7f5f859bea581a21c1d05430d1bbdcf2b393599b1cdce77"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:99b7920e7165be5a9e9a3a7f1b680f06f68ff0d0328ff4079e5163990d046767"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0dc49f42422163efb7e6f1df2636fe3db72713f6cd94688e339dbe33fe06d61d"}, + {file = "aiohttp-3.11.10-cp310-cp310-win32.whl", hash = "sha256:40d1c7a7f750b5648642586ba7206999650208dbe5afbcc5284bcec6579c9b91"}, + {file = "aiohttp-3.11.10-cp310-cp310-win_amd64.whl", hash = "sha256:68ff6f48b51bd78ea92b31079817aff539f6c8fc80b6b8d6ca347d7c02384e33"}, + {file = "aiohttp-3.11.10-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:77c4aa15a89847b9891abf97f3d4048f3c2d667e00f8a623c89ad2dccee6771b"}, + {file = "aiohttp-3.11.10-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:909af95a72cedbefe5596f0bdf3055740f96c1a4baa0dd11fd74ca4de0b4e3f1"}, + {file = "aiohttp-3.11.10-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:386fbe79863eb564e9f3615b959e28b222259da0c48fd1be5929ac838bc65683"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3de34936eb1a647aa919655ff8d38b618e9f6b7f250cc19a57a4bf7fd2062b6d"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0c9527819b29cd2b9f52033e7fb9ff08073df49b4799c89cb5754624ecd98299"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:65a96e3e03300b41f261bbfd40dfdbf1c301e87eab7cd61c054b1f2e7c89b9e8"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:98f5635f7b74bcd4f6f72fcd85bea2154b323a9f05226a80bc7398d0c90763b0"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:03b6002e20938fc6ee0918c81d9e776bebccc84690e2b03ed132331cca065ee5"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:6362cc6c23c08d18ddbf0e8c4d5159b5df74fea1a5278ff4f2c79aed3f4e9f46"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:3691ed7726fef54e928fe26344d930c0c8575bc968c3e239c2e1a04bd8cf7838"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:31d5093d3acd02b31c649d3a69bb072d539d4c7659b87caa4f6d2bcf57c2fa2b"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:8b3cf2dc0f0690a33f2d2b2cb15db87a65f1c609f53c37e226f84edb08d10f52"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:fbbaea811a2bba171197b08eea288b9402faa2bab2ba0858eecdd0a4105753a3"}, + {file = "aiohttp-3.11.10-cp311-cp311-win32.whl", hash = "sha256:4b2c7ac59c5698a7a8207ba72d9e9c15b0fc484a560be0788b31312c2c5504e4"}, + {file = "aiohttp-3.11.10-cp311-cp311-win_amd64.whl", hash = "sha256:974d3a2cce5fcfa32f06b13ccc8f20c6ad9c51802bb7f829eae8a1845c4019ec"}, + {file = "aiohttp-3.11.10-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:b78f053a7ecfc35f0451d961dacdc671f4bcbc2f58241a7c820e9d82559844cf"}, + {file = "aiohttp-3.11.10-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:ab7485222db0959a87fbe8125e233b5a6f01f4400785b36e8a7878170d8c3138"}, + {file = "aiohttp-3.11.10-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:cf14627232dfa8730453752e9cdc210966490992234d77ff90bc8dc0dce361d5"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:076bc454a7e6fd646bc82ea7f98296be0b1219b5e3ef8a488afbdd8e81fbac50"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:482cafb7dc886bebeb6c9ba7925e03591a62ab34298ee70d3dd47ba966370d2c"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bf3d1a519a324af764a46da4115bdbd566b3c73fb793ffb97f9111dbc684fc4d"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:24213ba85a419103e641e55c27dc7ff03536c4873470c2478cce3311ba1eee7b"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b99acd4730ad1b196bfb03ee0803e4adac371ae8efa7e1cbc820200fc5ded109"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:14cdb5a9570be5a04eec2ace174a48ae85833c2aadc86de68f55541f66ce42ab"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:7e97d622cb083e86f18317282084bc9fbf261801b0192c34fe4b1febd9f7ae69"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:012f176945af138abc10c4a48743327a92b4ca9adc7a0e078077cdb5dbab7be0"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:44224d815853962f48fe124748227773acd9686eba6dc102578defd6fc99e8d9"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c87bf31b7fdab94ae3adbe4a48e711bfc5f89d21cf4c197e75561def39e223bc"}, + {file = "aiohttp-3.11.10-cp312-cp312-win32.whl", hash = "sha256:06a8e2ee1cbac16fe61e51e0b0c269400e781b13bcfc33f5425912391a542985"}, + {file = "aiohttp-3.11.10-cp312-cp312-win_amd64.whl", hash = "sha256:be2b516f56ea883a3e14dda17059716593526e10fb6303189aaf5503937db408"}, + {file = "aiohttp-3.11.10-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:8cc5203b817b748adccb07f36390feb730b1bc5f56683445bfe924fc270b8816"}, + {file = "aiohttp-3.11.10-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:5ef359ebc6949e3a34c65ce20230fae70920714367c63afd80ea0c2702902ccf"}, + {file = "aiohttp-3.11.10-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:9bca390cb247dbfaec3c664326e034ef23882c3f3bfa5fbf0b56cad0320aaca5"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:811f23b3351ca532af598405db1093f018edf81368e689d1b508c57dcc6b6a32"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ddf5f7d877615f6a1e75971bfa5ac88609af3b74796ff3e06879e8422729fd01"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6ab29b8a0beb6f8eaf1e5049252cfe74adbaafd39ba91e10f18caeb0e99ffb34"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c49a76c1038c2dd116fa443eba26bbb8e6c37e924e2513574856de3b6516be99"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7f3dc0e330575f5b134918976a645e79adf333c0a1439dcf6899a80776c9ab39"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:efb15a17a12497685304b2d976cb4939e55137df7b09fa53f1b6a023f01fcb4e"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:db1d0b28fcb7f1d35600150c3e4b490775251dea70f894bf15c678fdd84eda6a"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:15fccaf62a4889527539ecb86834084ecf6e9ea70588efde86e8bc775e0e7542"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:593c114a2221444f30749cc5e5f4012488f56bd14de2af44fe23e1e9894a9c60"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:7852bbcb4d0d2f0c4d583f40c3bc750ee033265d80598d0f9cb6f372baa6b836"}, + {file = "aiohttp-3.11.10-cp313-cp313-win32.whl", hash = "sha256:65e55ca7debae8faaffee0ebb4b47a51b4075f01e9b641c31e554fd376595c6c"}, + {file = "aiohttp-3.11.10-cp313-cp313-win_amd64.whl", hash = "sha256:beb39a6d60a709ae3fb3516a1581777e7e8b76933bb88c8f4420d875bb0267c6"}, + {file = "aiohttp-3.11.10-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:0580f2e12de2138f34debcd5d88894786453a76e98febaf3e8fe5db62d01c9bf"}, + {file = "aiohttp-3.11.10-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a55d2ad345684e7c3dd2c20d2f9572e9e1d5446d57200ff630e6ede7612e307f"}, + {file = "aiohttp-3.11.10-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:04814571cb72d65a6899db6099e377ed00710bf2e3eafd2985166f2918beaf59"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e44a9a3c053b90c6f09b1bb4edd880959f5328cf63052503f892c41ea786d99f"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:502a1464ccbc800b4b1995b302efaf426e8763fadf185e933c2931df7db9a199"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:613e5169f8ae77b1933e42e418a95931fb4867b2991fc311430b15901ed67079"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4cca22a61b7fe45da8fc73c3443150c3608750bbe27641fc7558ec5117b27fdf"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:86a5dfcc39309470bd7b68c591d84056d195428d5d2e0b5ccadfbaf25b026ebc"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:77ae58586930ee6b2b6f696c82cf8e78c8016ec4795c53e36718365f6959dc82"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:78153314f26d5abef3239b4a9af20c229c6f3ecb97d4c1c01b22c4f87669820c"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:98283b94cc0e11c73acaf1c9698dea80c830ca476492c0fe2622bd931f34b487"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:53bf2097e05c2accc166c142a2090e4c6fd86581bde3fd9b2d3f9e93dda66ac1"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:c5532f0441fc09c119e1dca18fbc0687e64fbeb45aa4d6a87211ceaee50a74c4"}, + {file = "aiohttp-3.11.10-cp39-cp39-win32.whl", hash = "sha256:47ad15a65fb41c570cd0ad9a9ff8012489e68176e7207ec7b82a0940dddfd8be"}, + {file = "aiohttp-3.11.10-cp39-cp39-win_amd64.whl", hash = "sha256:c6b9e6d7e41656d78e37ce754813fa44b455c3d0d0dced2a047def7dc5570b74"}, + {file = "aiohttp-3.11.10.tar.gz", hash = "sha256:b1fc6b45010a8d0ff9e88f9f2418c6fd408c99c211257334aff41597ebece42e"}, ] [package.dependencies] @@ -158,24 +158,24 @@ files = [ [[package]] name = "anyio" -version = "4.6.2.post1" +version = "4.7.0" description = "High level compatibility layer for multiple asynchronous event loop implementations" optional = true python-versions = ">=3.9" files = [ - {file = "anyio-4.6.2.post1-py3-none-any.whl", hash = "sha256:6d170c36fba3bdd840c73d3868c1e777e33676a69c3a72cf0a0d5d6d8009b61d"}, - {file = "anyio-4.6.2.post1.tar.gz", hash = "sha256:4c8bc31ccdb51c7f7bd251f51c609e038d63e34219b44aa86e47576389880b4c"}, + {file = "anyio-4.7.0-py3-none-any.whl", hash = "sha256:ea60c3723ab42ba6fff7e8ccb0488c898ec538ff4df1f1d5e642c3601d07e352"}, + {file = "anyio-4.7.0.tar.gz", hash = "sha256:2f834749c602966b7d456a7567cafcb309f96482b5081d14ac93ccd457f9dd48"}, ] [package.dependencies] exceptiongroup = {version = ">=1.0.2", markers = "python_version < \"3.11\""} idna = ">=2.8" sniffio = ">=1.1" -typing-extensions = {version = ">=4.1", markers = "python_version < \"3.11\""} +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)", "pytest-mock (>=3.6.1)", "trustme", "truststore (>=0.9.1)", "uvloop (>=0.21.0b1)"] +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)", "pytest-mock (>=3.6.1)", "trustme", "truststore (>=0.9.1)", "uvloop (>=0.21)"] trio = ["trio (>=0.26.1)"] [[package]] @@ -674,73 +674,73 @@ test-no-images = ["pytest", "pytest-cov", "pytest-rerunfailures", "pytest-xdist" [[package]] name = "coverage" -version = "7.6.8" +version = "7.6.9" description = "Code coverage measurement for Python" optional = false python-versions = ">=3.9" files = [ - {file = "coverage-7.6.8-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b39e6011cd06822eb964d038d5dff5da5d98652b81f5ecd439277b32361a3a50"}, - {file = "coverage-7.6.8-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:63c19702db10ad79151a059d2d6336fe0c470f2e18d0d4d1a57f7f9713875dcf"}, - {file = "coverage-7.6.8-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3985b9be361d8fb6b2d1adc9924d01dec575a1d7453a14cccd73225cb79243ee"}, - {file = "coverage-7.6.8-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:644ec81edec0f4ad17d51c838a7d01e42811054543b76d4ba2c5d6af741ce2a6"}, - {file = "coverage-7.6.8-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1f188a2402f8359cf0c4b1fe89eea40dc13b52e7b4fd4812450da9fcd210181d"}, - {file = "coverage-7.6.8-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:e19122296822deafce89a0c5e8685704c067ae65d45e79718c92df7b3ec3d331"}, - {file = "coverage-7.6.8-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:13618bed0c38acc418896005732e565b317aa9e98d855a0e9f211a7ffc2d6638"}, - {file = "coverage-7.6.8-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:193e3bffca48ad74b8c764fb4492dd875038a2f9925530cb094db92bb5e47bed"}, - {file = "coverage-7.6.8-cp310-cp310-win32.whl", hash = "sha256:3988665ee376abce49613701336544041f2117de7b7fbfe91b93d8ff8b151c8e"}, - {file = "coverage-7.6.8-cp310-cp310-win_amd64.whl", hash = "sha256:f56f49b2553d7dd85fd86e029515a221e5c1f8cb3d9c38b470bc38bde7b8445a"}, - {file = "coverage-7.6.8-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:86cffe9c6dfcfe22e28027069725c7f57f4b868a3f86e81d1c62462764dc46d4"}, - {file = "coverage-7.6.8-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d82ab6816c3277dc962cfcdc85b1efa0e5f50fb2c449432deaf2398a2928ab94"}, - {file = "coverage-7.6.8-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:13690e923a3932e4fad4c0ebfb9cb5988e03d9dcb4c5150b5fcbf58fd8bddfc4"}, - {file = "coverage-7.6.8-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4be32da0c3827ac9132bb488d331cb32e8d9638dd41a0557c5569d57cf22c9c1"}, - {file = "coverage-7.6.8-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:44e6c85bbdc809383b509d732b06419fb4544dca29ebe18480379633623baafb"}, - {file = "coverage-7.6.8-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:768939f7c4353c0fac2f7c37897e10b1414b571fd85dd9fc49e6a87e37a2e0d8"}, - {file = "coverage-7.6.8-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e44961e36cb13c495806d4cac67640ac2866cb99044e210895b506c26ee63d3a"}, - {file = "coverage-7.6.8-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:3ea8bb1ab9558374c0ab591783808511d135a833c3ca64a18ec927f20c4030f0"}, - {file = "coverage-7.6.8-cp311-cp311-win32.whl", hash = "sha256:629a1ba2115dce8bf75a5cce9f2486ae483cb89c0145795603d6554bdc83e801"}, - {file = "coverage-7.6.8-cp311-cp311-win_amd64.whl", hash = "sha256:fb9fc32399dca861584d96eccd6c980b69bbcd7c228d06fb74fe53e007aa8ef9"}, - {file = "coverage-7.6.8-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:e683e6ecc587643f8cde8f5da6768e9d165cd31edf39ee90ed7034f9ca0eefee"}, - {file = "coverage-7.6.8-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1defe91d41ce1bd44b40fabf071e6a01a5aa14de4a31b986aa9dfd1b3e3e414a"}, - {file = "coverage-7.6.8-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7ad66e8e50225ebf4236368cc43c37f59d5e6728f15f6e258c8639fa0dd8e6d"}, - {file = "coverage-7.6.8-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3fe47da3e4fda5f1abb5709c156eca207eacf8007304ce3019eb001e7a7204cb"}, - {file = "coverage-7.6.8-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:202a2d645c5a46b84992f55b0a3affe4f0ba6b4c611abec32ee88358db4bb649"}, - {file = "coverage-7.6.8-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:4674f0daa1823c295845b6a740d98a840d7a1c11df00d1fd62614545c1583787"}, - {file = "coverage-7.6.8-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:74610105ebd6f33d7c10f8907afed696e79c59e3043c5f20eaa3a46fddf33b4c"}, - {file = "coverage-7.6.8-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:37cda8712145917105e07aab96388ae76e787270ec04bcb9d5cc786d7cbb8443"}, - {file = "coverage-7.6.8-cp312-cp312-win32.whl", hash = "sha256:9e89d5c8509fbd6c03d0dd1972925b22f50db0792ce06324ba069f10787429ad"}, - {file = "coverage-7.6.8-cp312-cp312-win_amd64.whl", hash = "sha256:379c111d3558272a2cae3d8e57e6b6e6f4fe652905692d54bad5ea0ca37c5ad4"}, - {file = "coverage-7.6.8-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:0b0c69f4f724c64dfbfe79f5dfb503b42fe6127b8d479b2677f2b227478db2eb"}, - {file = "coverage-7.6.8-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:c15b32a7aca8038ed7644f854bf17b663bc38e1671b5d6f43f9a2b2bd0c46f63"}, - {file = "coverage-7.6.8-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:63068a11171e4276f6ece913bde059e77c713b48c3a848814a6537f35afb8365"}, - {file = "coverage-7.6.8-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6f4548c5ead23ad13fb7a2c8ea541357474ec13c2b736feb02e19a3085fac002"}, - {file = "coverage-7.6.8-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b4b4299dd0d2c67caaaf286d58aef5e75b125b95615dda4542561a5a566a1e3"}, - {file = "coverage-7.6.8-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:c9ebfb2507751f7196995142f057d1324afdab56db1d9743aab7f50289abd022"}, - {file = "coverage-7.6.8-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:c1b4474beee02ede1eef86c25ad4600a424fe36cff01a6103cb4533c6bf0169e"}, - {file = "coverage-7.6.8-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:d9fd2547e6decdbf985d579cf3fc78e4c1d662b9b0ff7cc7862baaab71c9cc5b"}, - {file = "coverage-7.6.8-cp313-cp313-win32.whl", hash = "sha256:8aae5aea53cbfe024919715eca696b1a3201886ce83790537d1c3668459c7146"}, - {file = "coverage-7.6.8-cp313-cp313-win_amd64.whl", hash = "sha256:ae270e79f7e169ccfe23284ff5ea2d52a6f401dc01b337efb54b3783e2ce3f28"}, - {file = "coverage-7.6.8-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:de38add67a0af869b0d79c525d3e4588ac1ffa92f39116dbe0ed9753f26eba7d"}, - {file = "coverage-7.6.8-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:b07c25d52b1c16ce5de088046cd2432b30f9ad5e224ff17c8f496d9cb7d1d451"}, - {file = "coverage-7.6.8-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:62a66ff235e4c2e37ed3b6104d8b478d767ff73838d1222132a7a026aa548764"}, - {file = "coverage-7.6.8-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:09b9f848b28081e7b975a3626e9081574a7b9196cde26604540582da60235fdf"}, - {file = "coverage-7.6.8-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:093896e530c38c8e9c996901858ac63f3d4171268db2c9c8b373a228f459bbc5"}, - {file = "coverage-7.6.8-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:9a7b8ac36fd688c8361cbc7bf1cb5866977ece6e0b17c34aa0df58bda4fa18a4"}, - {file = "coverage-7.6.8-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:38c51297b35b3ed91670e1e4efb702b790002e3245a28c76e627478aa3c10d83"}, - {file = "coverage-7.6.8-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:2e4e0f60cb4bd7396108823548e82fdab72d4d8a65e58e2c19bbbc2f1e2bfa4b"}, - {file = "coverage-7.6.8-cp313-cp313t-win32.whl", hash = "sha256:6535d996f6537ecb298b4e287a855f37deaf64ff007162ec0afb9ab8ba3b8b71"}, - {file = "coverage-7.6.8-cp313-cp313t-win_amd64.whl", hash = "sha256:c79c0685f142ca53256722a384540832420dff4ab15fec1863d7e5bc8691bdcc"}, - {file = "coverage-7.6.8-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3ac47fa29d8d41059ea3df65bd3ade92f97ee4910ed638e87075b8e8ce69599e"}, - {file = "coverage-7.6.8-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:24eda3a24a38157eee639ca9afe45eefa8d2420d49468819ac5f88b10de84f4c"}, - {file = "coverage-7.6.8-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e4c81ed2820b9023a9a90717020315e63b17b18c274a332e3b6437d7ff70abe0"}, - {file = "coverage-7.6.8-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:bd55f8fc8fa494958772a2a7302b0354ab16e0b9272b3c3d83cdb5bec5bd1779"}, - {file = "coverage-7.6.8-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f39e2f3530ed1626c66e7493be7a8423b023ca852aacdc91fb30162c350d2a92"}, - {file = "coverage-7.6.8-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:716a78a342679cd1177bc8c2fe957e0ab91405bd43a17094324845200b2fddf4"}, - {file = "coverage-7.6.8-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:177f01eeaa3aee4a5ffb0d1439c5952b53d5010f86e9d2667963e632e30082cc"}, - {file = "coverage-7.6.8-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:912e95017ff51dc3d7b6e2be158dedc889d9a5cc3382445589ce554f1a34c0ea"}, - {file = "coverage-7.6.8-cp39-cp39-win32.whl", hash = "sha256:4db3ed6a907b555e57cc2e6f14dc3a4c2458cdad8919e40b5357ab9b6db6c43e"}, - {file = "coverage-7.6.8-cp39-cp39-win_amd64.whl", hash = "sha256:428ac484592f780e8cd7b6b14eb568f7c85460c92e2a37cb0c0e5186e1a0d076"}, - {file = "coverage-7.6.8-pp39.pp310-none-any.whl", hash = "sha256:5c52a036535d12590c32c49209e79cabaad9f9ad8aa4cbd875b68c4d67a9cbce"}, - {file = "coverage-7.6.8.tar.gz", hash = "sha256:8b2b8503edb06822c86d82fa64a4a5cb0760bb8f31f26e138ec743f422f37cfc"}, + {file = "coverage-7.6.9-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:85d9636f72e8991a1706b2b55b06c27545448baf9f6dbf51c4004609aacd7dcb"}, + {file = "coverage-7.6.9-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:608a7fd78c67bee8936378299a6cb9f5149bb80238c7a566fc3e6717a4e68710"}, + {file = "coverage-7.6.9-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:96d636c77af18b5cb664ddf12dab9b15a0cfe9c0bde715da38698c8cea748bfa"}, + {file = "coverage-7.6.9-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d75cded8a3cff93da9edc31446872d2997e327921d8eed86641efafd350e1df1"}, + {file = "coverage-7.6.9-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7b15f589593110ae767ce997775d645b47e5cbbf54fd322f8ebea6277466cec"}, + {file = "coverage-7.6.9-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:44349150f6811b44b25574839b39ae35291f6496eb795b7366fef3bd3cf112d3"}, + {file = "coverage-7.6.9-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:d891c136b5b310d0e702e186d70cd16d1119ea8927347045124cb286b29297e5"}, + {file = "coverage-7.6.9-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:db1dab894cc139f67822a92910466531de5ea6034ddfd2b11c0d4c6257168073"}, + {file = "coverage-7.6.9-cp310-cp310-win32.whl", hash = "sha256:41ff7b0da5af71a51b53f501a3bac65fb0ec311ebed1632e58fc6107f03b9198"}, + {file = "coverage-7.6.9-cp310-cp310-win_amd64.whl", hash = "sha256:35371f8438028fdccfaf3570b31d98e8d9eda8bb1d6ab9473f5a390969e98717"}, + {file = "coverage-7.6.9-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:932fc826442132dde42ee52cf66d941f581c685a6313feebed358411238f60f9"}, + {file = "coverage-7.6.9-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:085161be5f3b30fd9b3e7b9a8c301f935c8313dcf928a07b116324abea2c1c2c"}, + {file = "coverage-7.6.9-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ccc660a77e1c2bf24ddbce969af9447a9474790160cfb23de6be4fa88e3951c7"}, + {file = "coverage-7.6.9-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c69e42c892c018cd3c8d90da61d845f50a8243062b19d228189b0224150018a9"}, + {file = "coverage-7.6.9-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0824a28ec542a0be22f60c6ac36d679e0e262e5353203bea81d44ee81fe9c6d4"}, + {file = "coverage-7.6.9-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:4401ae5fc52ad8d26d2a5d8a7428b0f0c72431683f8e63e42e70606374c311a1"}, + {file = "coverage-7.6.9-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:98caba4476a6c8d59ec1eb00c7dd862ba9beca34085642d46ed503cc2d440d4b"}, + {file = "coverage-7.6.9-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ee5defd1733fd6ec08b168bd4f5387d5b322f45ca9e0e6c817ea6c4cd36313e3"}, + {file = "coverage-7.6.9-cp311-cp311-win32.whl", hash = "sha256:f2d1ec60d6d256bdf298cb86b78dd715980828f50c46701abc3b0a2b3f8a0dc0"}, + {file = "coverage-7.6.9-cp311-cp311-win_amd64.whl", hash = "sha256:0d59fd927b1f04de57a2ba0137166d31c1a6dd9e764ad4af552912d70428c92b"}, + {file = "coverage-7.6.9-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:99e266ae0b5d15f1ca8d278a668df6f51cc4b854513daab5cae695ed7b721cf8"}, + {file = "coverage-7.6.9-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:9901d36492009a0a9b94b20e52ebfc8453bf49bb2b27bca2c9706f8b4f5a554a"}, + {file = "coverage-7.6.9-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:abd3e72dd5b97e3af4246cdada7738ef0e608168de952b837b8dd7e90341f015"}, + {file = "coverage-7.6.9-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ff74026a461eb0660366fb01c650c1d00f833a086b336bdad7ab00cc952072b3"}, + {file = "coverage-7.6.9-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:65dad5a248823a4996724a88eb51d4b31587aa7aa428562dbe459c684e5787ae"}, + {file = "coverage-7.6.9-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:22be16571504c9ccea919fcedb459d5ab20d41172056206eb2994e2ff06118a4"}, + {file = "coverage-7.6.9-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f957943bc718b87144ecaee70762bc2bc3f1a7a53c7b861103546d3a403f0a6"}, + {file = "coverage-7.6.9-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:0ae1387db4aecb1f485fb70a6c0148c6cdaebb6038f1d40089b1fc84a5db556f"}, + {file = "coverage-7.6.9-cp312-cp312-win32.whl", hash = "sha256:1a330812d9cc7ac2182586f6d41b4d0fadf9be9049f350e0efb275c8ee8eb692"}, + {file = "coverage-7.6.9-cp312-cp312-win_amd64.whl", hash = "sha256:b12c6b18269ca471eedd41c1b6a1065b2f7827508edb9a7ed5555e9a56dcfc97"}, + {file = "coverage-7.6.9-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:899b8cd4781c400454f2f64f7776a5d87bbd7b3e7f7bda0cb18f857bb1334664"}, + {file = "coverage-7.6.9-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:61f70dc68bd36810972e55bbbe83674ea073dd1dcc121040a08cdf3416c5349c"}, + {file = "coverage-7.6.9-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8a289d23d4c46f1a82d5db4abeb40b9b5be91731ee19a379d15790e53031c014"}, + {file = "coverage-7.6.9-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7e216d8044a356fc0337c7a2a0536d6de07888d7bcda76febcb8adc50bdbbd00"}, + {file = "coverage-7.6.9-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3c026eb44f744acaa2bda7493dad903aa5bf5fc4f2554293a798d5606710055d"}, + {file = "coverage-7.6.9-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:e77363e8425325384f9d49272c54045bbed2f478e9dd698dbc65dbc37860eb0a"}, + {file = "coverage-7.6.9-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:777abfab476cf83b5177b84d7486497e034eb9eaea0d746ce0c1268c71652077"}, + {file = "coverage-7.6.9-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:447af20e25fdbe16f26e84eb714ba21d98868705cb138252d28bc400381f6ffb"}, + {file = "coverage-7.6.9-cp313-cp313-win32.whl", hash = "sha256:d872ec5aeb086cbea771c573600d47944eea2dcba8be5f3ee649bfe3cb8dc9ba"}, + {file = "coverage-7.6.9-cp313-cp313-win_amd64.whl", hash = "sha256:fd1213c86e48dfdc5a0cc676551db467495a95a662d2396ecd58e719191446e1"}, + {file = "coverage-7.6.9-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:ba9e7484d286cd5a43744e5f47b0b3fb457865baf07bafc6bee91896364e1419"}, + {file = "coverage-7.6.9-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e5ea1cf0872ee455c03e5674b5bca5e3e68e159379c1af0903e89f5eba9ccc3a"}, + {file = "coverage-7.6.9-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2d10e07aa2b91835d6abec555ec8b2733347956991901eea6ffac295f83a30e4"}, + {file = "coverage-7.6.9-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:13a9e2d3ee855db3dd6ea1ba5203316a1b1fd8eaeffc37c5b54987e61e4194ae"}, + {file = "coverage-7.6.9-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c38bf15a40ccf5619fa2fe8f26106c7e8e080d7760aeccb3722664c8656b030"}, + {file = "coverage-7.6.9-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:d5275455b3e4627c8e7154feaf7ee0743c2e7af82f6e3b561967b1cca755a0be"}, + {file = "coverage-7.6.9-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:8f8770dfc6e2c6a2d4569f411015c8d751c980d17a14b0530da2d7f27ffdd88e"}, + {file = "coverage-7.6.9-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:8d2dfa71665a29b153a9681edb1c8d9c1ea50dfc2375fb4dac99ea7e21a0bcd9"}, + {file = "coverage-7.6.9-cp313-cp313t-win32.whl", hash = "sha256:5e6b86b5847a016d0fbd31ffe1001b63355ed309651851295315031ea7eb5a9b"}, + {file = "coverage-7.6.9-cp313-cp313t-win_amd64.whl", hash = "sha256:97ddc94d46088304772d21b060041c97fc16bdda13c6c7f9d8fcd8d5ae0d8611"}, + {file = "coverage-7.6.9-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:adb697c0bd35100dc690de83154627fbab1f4f3c0386df266dded865fc50a902"}, + {file = "coverage-7.6.9-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:be57b6d56e49c2739cdf776839a92330e933dd5e5d929966fbbd380c77f060be"}, + {file = "coverage-7.6.9-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1592791f8204ae9166de22ba7e6705fa4ebd02936c09436a1bb85aabca3e599"}, + {file = "coverage-7.6.9-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4e12ae8cc979cf83d258acb5e1f1cf2f3f83524d1564a49d20b8bec14b637f08"}, + {file = "coverage-7.6.9-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb5555cff66c4d3d6213a296b360f9e1a8e323e74e0426b6c10ed7f4d021e464"}, + {file = "coverage-7.6.9-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:b9389a429e0e5142e69d5bf4a435dd688c14478a19bb901735cdf75e57b13845"}, + {file = "coverage-7.6.9-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:592ac539812e9b46046620341498caf09ca21023c41c893e1eb9dbda00a70cbf"}, + {file = "coverage-7.6.9-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:a27801adef24cc30871da98a105f77995e13a25a505a0161911f6aafbd66e678"}, + {file = "coverage-7.6.9-cp39-cp39-win32.whl", hash = "sha256:8e3c3e38930cfb729cb8137d7f055e5a473ddaf1217966aa6238c88bd9fd50e6"}, + {file = "coverage-7.6.9-cp39-cp39-win_amd64.whl", hash = "sha256:e28bf44afa2b187cc9f41749138a64435bf340adfcacb5b2290c070ce99839d4"}, + {file = "coverage-7.6.9-pp39.pp310-none-any.whl", hash = "sha256:f3ca78518bc6bc92828cd11867b121891d75cae4ea9e908d72030609b996db1b"}, + {file = "coverage-7.6.9.tar.gz", hash = "sha256:4a8d8977b0c6ef5aeadcb644da9e69ae0dcfe66ec7f368c89c72e058bd71164d"}, ] [package.dependencies] @@ -1074,61 +1074,61 @@ pyflakes = ">=3.1.0,<3.2.0" [[package]] name = "fonttools" -version = "4.55.0" +version = "4.55.2" description = "Tools to manipulate font files" optional = true python-versions = ">=3.8" files = [ - {file = "fonttools-4.55.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:51c029d4c0608a21a3d3d169dfc3fb776fde38f00b35ca11fdab63ba10a16f61"}, - {file = "fonttools-4.55.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bca35b4e411362feab28e576ea10f11268b1aeed883b9f22ed05675b1e06ac69"}, - {file = "fonttools-4.55.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9ce4ba6981e10f7e0ccff6348e9775ce25ffadbee70c9fd1a3737e3e9f5fa74f"}, - {file = "fonttools-4.55.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:31d00f9852a6051dac23294a4cf2df80ced85d1d173a61ba90a3d8f5abc63c60"}, - {file = "fonttools-4.55.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:e198e494ca6e11f254bac37a680473a311a88cd40e58f9cc4dc4911dfb686ec6"}, - {file = "fonttools-4.55.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:7208856f61770895e79732e1dcbe49d77bd5783adf73ae35f87fcc267df9db81"}, - {file = "fonttools-4.55.0-cp310-cp310-win32.whl", hash = "sha256:e7e6a352ff9e46e8ef8a3b1fe2c4478f8a553e1b5a479f2e899f9dc5f2055880"}, - {file = "fonttools-4.55.0-cp310-cp310-win_amd64.whl", hash = "sha256:636caaeefe586d7c84b5ee0734c1a5ab2dae619dc21c5cf336f304ddb8f6001b"}, - {file = "fonttools-4.55.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:fa34aa175c91477485c44ddfbb51827d470011e558dfd5c7309eb31bef19ec51"}, - {file = "fonttools-4.55.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:37dbb3fdc2ef7302d3199fb12468481cbebaee849e4b04bc55b77c24e3c49189"}, - {file = "fonttools-4.55.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b5263d8e7ef3c0ae87fbce7f3ec2f546dc898d44a337e95695af2cd5ea21a967"}, - {file = "fonttools-4.55.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f307f6b5bf9e86891213b293e538d292cd1677e06d9faaa4bf9c086ad5f132f6"}, - {file = "fonttools-4.55.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:f0a4b52238e7b54f998d6a56b46a2c56b59c74d4f8a6747fb9d4042190f37cd3"}, - {file = "fonttools-4.55.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:3e569711464f777a5d4ef522e781dc33f8095ab5efd7548958b36079a9f2f88c"}, - {file = "fonttools-4.55.0-cp311-cp311-win32.whl", hash = "sha256:2b3ab90ec0f7b76c983950ac601b58949f47aca14c3f21eed858b38d7ec42b05"}, - {file = "fonttools-4.55.0-cp311-cp311-win_amd64.whl", hash = "sha256:aa046f6a63bb2ad521004b2769095d4c9480c02c1efa7d7796b37826508980b6"}, - {file = "fonttools-4.55.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:838d2d8870f84fc785528a692e724f2379d5abd3fc9dad4d32f91cf99b41e4a7"}, - {file = "fonttools-4.55.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:f46b863d74bab7bb0d395f3b68d3f52a03444964e67ce5c43ce43a75efce9246"}, - {file = "fonttools-4.55.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:33b52a9cfe4e658e21b1f669f7309b4067910321757fec53802ca8f6eae96a5a"}, - {file = "fonttools-4.55.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:732a9a63d6ea4a81b1b25a1f2e5e143761b40c2e1b79bb2b68e4893f45139a40"}, - {file = "fonttools-4.55.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7dd91ac3fcb4c491bb4763b820bcab6c41c784111c24172616f02f4bc227c17d"}, - {file = "fonttools-4.55.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:1f0e115281a32ff532118aa851ef497a1b7cda617f4621c1cdf81ace3e36fb0c"}, - {file = "fonttools-4.55.0-cp312-cp312-win32.whl", hash = "sha256:6c99b5205844f48a05cb58d4a8110a44d3038c67ed1d79eb733c4953c628b0f6"}, - {file = "fonttools-4.55.0-cp312-cp312-win_amd64.whl", hash = "sha256:f8c8c76037d05652510ae45be1cd8fb5dd2fd9afec92a25374ac82255993d57c"}, - {file = "fonttools-4.55.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:8118dc571921dc9e4b288d9cb423ceaf886d195a2e5329cc427df82bba872cd9"}, - {file = "fonttools-4.55.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:01124f2ca6c29fad4132d930da69158d3f49b2350e4a779e1efbe0e82bd63f6c"}, - {file = "fonttools-4.55.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:81ffd58d2691f11f7c8438796e9f21c374828805d33e83ff4b76e4635633674c"}, - {file = "fonttools-4.55.0-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5435e5f1eb893c35c2bc2b9cd3c9596b0fcb0a59e7a14121562986dd4c47b8dd"}, - {file = "fonttools-4.55.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:d12081729280c39d001edd0f4f06d696014c26e6e9a0a55488fabc37c28945e4"}, - {file = "fonttools-4.55.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:a7ad1f1b98ab6cb927ab924a38a8649f1ffd7525c75fe5b594f5dab17af70e18"}, - {file = "fonttools-4.55.0-cp313-cp313-win32.whl", hash = "sha256:abe62987c37630dca69a104266277216de1023cf570c1643bb3a19a9509e7a1b"}, - {file = "fonttools-4.55.0-cp313-cp313-win_amd64.whl", hash = "sha256:2863555ba90b573e4201feaf87a7e71ca3b97c05aa4d63548a4b69ea16c9e998"}, - {file = "fonttools-4.55.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:00f7cf55ad58a57ba421b6a40945b85ac7cc73094fb4949c41171d3619a3a47e"}, - {file = "fonttools-4.55.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:f27526042efd6f67bfb0cc2f1610fa20364396f8b1fc5edb9f45bb815fb090b2"}, - {file = "fonttools-4.55.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e8e67974326af6a8879dc2a4ec63ab2910a1c1a9680ccd63e4a690950fceddbe"}, - {file = "fonttools-4.55.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:61dc0a13451143c5e987dec5254d9d428f3c2789a549a7cf4f815b63b310c1cc"}, - {file = "fonttools-4.55.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:b2e526b325a903868c62155a6a7e24df53f6ce4c5c3160214d8fe1be2c41b478"}, - {file = "fonttools-4.55.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:b7ef9068a1297714e6fefe5932c33b058aa1d45a2b8be32a4c6dee602ae22b5c"}, - {file = "fonttools-4.55.0-cp38-cp38-win32.whl", hash = "sha256:55718e8071be35dff098976bc249fc243b58efa263768c611be17fe55975d40a"}, - {file = "fonttools-4.55.0-cp38-cp38-win_amd64.whl", hash = "sha256:553bd4f8cc327f310c20158e345e8174c8eed49937fb047a8bda51daf2c353c8"}, - {file = "fonttools-4.55.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:3f901cef813f7c318b77d1c5c14cf7403bae5cb977cede023e22ba4316f0a8f6"}, - {file = "fonttools-4.55.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8c9679fc0dd7e8a5351d321d8d29a498255e69387590a86b596a45659a39eb0d"}, - {file = "fonttools-4.55.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dd2820a8b632f3307ebb0bf57948511c2208e34a4939cf978333bc0a3f11f838"}, - {file = "fonttools-4.55.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:23bbbb49bec613a32ed1b43df0f2b172313cee690c2509f1af8fdedcf0a17438"}, - {file = "fonttools-4.55.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:a656652e1f5d55b9728937a7e7d509b73d23109cddd4e89ee4f49bde03b736c6"}, - {file = "fonttools-4.55.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:f50a1f455902208486fbca47ce33054208a4e437b38da49d6721ce2fef732fcf"}, - {file = "fonttools-4.55.0-cp39-cp39-win32.whl", hash = "sha256:161d1ac54c73d82a3cded44202d0218ab007fde8cf194a23d3dd83f7177a2f03"}, - {file = "fonttools-4.55.0-cp39-cp39-win_amd64.whl", hash = "sha256:ca7fd6987c68414fece41c96836e945e1f320cda56fc96ffdc16e54a44ec57a2"}, - {file = "fonttools-4.55.0-py3-none-any.whl", hash = "sha256:12db5888cd4dd3fcc9f0ee60c6edd3c7e1fd44b7dd0f31381ea03df68f8a153f"}, - {file = "fonttools-4.55.0.tar.gz", hash = "sha256:7636acc6ab733572d5e7eec922b254ead611f1cdad17be3f0be7418e8bfaca71"}, + {file = "fonttools-4.55.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:bef0f8603834643b1a6419d57902f18e7d950ec1a998fb70410635c598dc1a1e"}, + {file = "fonttools-4.55.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:944228b86d472612d3b48bcc83b31c25c2271e63fdc74539adfcfa7a96d487fb"}, + {file = "fonttools-4.55.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9f0e55f5da594b85f269cfbecd2f6bd3e07d0abba68870bc3f34854de4fa4678"}, + {file = "fonttools-4.55.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5b1a6e576db0c83c1b91925bf1363478c4bb968dbe8433147332fb5782ce6190"}, + {file = "fonttools-4.55.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:616368b15716781bc84df5c2191dc0540137aaef56c2771eb4b89b90933f347a"}, + {file = "fonttools-4.55.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:7bbae4f3915225c2c37670da68e2bf18a21206060ad31dfb95fec91ef641caa7"}, + {file = "fonttools-4.55.2-cp310-cp310-win32.whl", hash = "sha256:8b02b10648d69d67a7eb055f4d3eedf4a85deb22fb7a19fbd9acbae7c7538199"}, + {file = "fonttools-4.55.2-cp310-cp310-win_amd64.whl", hash = "sha256:bbea0ab841113ac8e8edde067e099b7288ffc6ac2dded538b131c2c0595d5f77"}, + {file = "fonttools-4.55.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:d34525e8141286fa976e14806639d32294bfb38d28bbdb5f6be9f46a1cd695a6"}, + {file = "fonttools-4.55.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:0ecd1c2b1c2ec46bb73685bc5473c72e16ed0930ef79bc2919ccadc43a99fb16"}, + {file = "fonttools-4.55.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9008438ad59e5a8e403a62fbefef2b2ff377eb3857d90a3f2a5f4d674ff441b2"}, + {file = "fonttools-4.55.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:131591ac8d7a47043aaf29581aba755ae151d46e49d2bf49608601efd71e8b4d"}, + {file = "fonttools-4.55.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:4c83381c3e3e3d9caa25527c4300543578341f21aae89e4fbbb4debdda8d82a2"}, + {file = "fonttools-4.55.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:42aca564b575252fd9954ed0d91d97a24de24289a16ce8ff74ed0bdf5ecebf11"}, + {file = "fonttools-4.55.2-cp311-cp311-win32.whl", hash = "sha256:c6457f650ebe15baa17fc06e256227f0a47f46f80f27ec5a0b00160de8dc2c13"}, + {file = "fonttools-4.55.2-cp311-cp311-win_amd64.whl", hash = "sha256:5cfa67414d7414442a5635ff634384101c54f53bb7b0e04aa6a61b013fcce194"}, + {file = "fonttools-4.55.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:18f082445b8fe5e91c53e6184f4c1c73f3f965c8bcc614c6cd6effd573ce6c1a"}, + {file = "fonttools-4.55.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:27c0f91adbbd706e8acd1db73e3e510118e62d0ffb651864567dccc5b2339f90"}, + {file = "fonttools-4.55.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3d8ccce035320d63dba0c35f52499322f5531dbe85bba1514c7cea26297e4c54"}, + {file = "fonttools-4.55.2-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:96e126df9615df214ec7f04bebcf60076297fbc10b75c777ce58b702d7708ffb"}, + {file = "fonttools-4.55.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:508ebb42956a7a931c4092dfa2d9b4ffd4f94cea09b8211199090d2bd082506b"}, + {file = "fonttools-4.55.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c1b9de46ef7b683d50400abf9f1578eaceee271ff51c36bf4b7366f2be29f498"}, + {file = "fonttools-4.55.2-cp312-cp312-win32.whl", hash = "sha256:2df61d9fc15199cc86dad29f64dd686874a3a52dda0c2d8597d21f509f95c332"}, + {file = "fonttools-4.55.2-cp312-cp312-win_amd64.whl", hash = "sha256:d337ec087da8216a828574aa0525d869df0a2ac217a2efc1890974ddd1fbc5b9"}, + {file = "fonttools-4.55.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:10aff204e2edee1d312fa595c06f201adf8d528a3b659cfb34cd47eceaaa6a26"}, + {file = "fonttools-4.55.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:09fe922a3eff181fd07dd724cdb441fb6b9fc355fd1c0f1aa79aca60faf1fbdd"}, + {file = "fonttools-4.55.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:487e1e8b524143a799bda0169c48b44a23a6027c1bb1957d5a172a7d3a1dd704"}, + {file = "fonttools-4.55.2-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9b1726872e09268bbedb14dc02e58b7ea31ecdd1204c6073eda4911746b44797"}, + {file = "fonttools-4.55.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:6fc88cfb58b0cd7b48718c3e61dd0d0a3ee8e2c86b973342967ce09fbf1db6d4"}, + {file = "fonttools-4.55.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:e857fe1859901ad8c5cab32e0eebc920adb09f413d2d73b74b677cf47b28590c"}, + {file = "fonttools-4.55.2-cp313-cp313-win32.whl", hash = "sha256:81ccd2b3a420b8050c7d9db3be0555d71662973b3ef2a1d921a2880b58957db8"}, + {file = "fonttools-4.55.2-cp313-cp313-win_amd64.whl", hash = "sha256:d559eb1744c7dcfa90ae60cb1a4b3595e898e48f4198738c321468c01180cd83"}, + {file = "fonttools-4.55.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:6b5917ef79cac8300b88fd6113003fd01bbbbea2ea060a27b95d8f77cb4c65c2"}, + {file = "fonttools-4.55.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:663eba5615d6abaaf616432354eb7ce951d518e43404371bcc2b0694ef21e8d6"}, + {file = "fonttools-4.55.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:803d5cef5fc47f44f5084d154aa3d6f069bb1b60e32390c225f897fa19b0f939"}, + {file = "fonttools-4.55.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8bc5f100de0173cc39102c0399bd6c3bd544bbdf224957933f10ee442d43cddd"}, + {file = "fonttools-4.55.2-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:3d9bbc1e380fdaf04ad9eabd8e3e6a4301eaf3487940893e9fd98537ea2e283b"}, + {file = "fonttools-4.55.2-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:42a9afedff07b6f75aa0f39b5e49922ac764580ef3efce035ca30284b2ee65c8"}, + {file = "fonttools-4.55.2-cp38-cp38-win32.whl", hash = "sha256:f1c76f423f1a241df08f87614364dff6e0b7ce23c962c1b74bd995ec7c0dad13"}, + {file = "fonttools-4.55.2-cp38-cp38-win_amd64.whl", hash = "sha256:25062b6ca03464dd5179fc2040fb19e03391b7cc49b9cc4f879312e638605c5c"}, + {file = "fonttools-4.55.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:d1100d8e665fe386a79cab59446992de881ea74d0d6c191bb988642692aa2421"}, + {file = "fonttools-4.55.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:dbdc251c5e472e5ae6bc816f9b82718b8e93ff7992e7331d6cf3562b96aa268e"}, + {file = "fonttools-4.55.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d0bf24d2b02dbc9376d795a63062632ff73e3e9e60c0229373f500aed7e86dd7"}, + {file = "fonttools-4.55.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d4ff250ed4ff05015dfd9cf2adf7570c7a383ca80f4d9732ac484a5ed0d8453c"}, + {file = "fonttools-4.55.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:44cf2a98aa661dbdeb8c03f5e405b074e2935196780bb729888639f5276067d9"}, + {file = "fonttools-4.55.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:22ef222740eb89d189bf0612eb98fbae592c61d7efeac51bfbc2a1592d469557"}, + {file = "fonttools-4.55.2-cp39-cp39-win32.whl", hash = "sha256:93f439ca27e55f585e7aaa04a74990acd983b5f2245e41d6b79f0a8b44e684d8"}, + {file = "fonttools-4.55.2-cp39-cp39-win_amd64.whl", hash = "sha256:627cf10d6f5af5bec6324c18a2670f134c29e1b7dce3fb62e8ef88baa6cba7a9"}, + {file = "fonttools-4.55.2-py3-none-any.whl", hash = "sha256:8e2d89fbe9b08d96e22c7a81ec04a4e8d8439c31223e2dc6f2f9fc8ff14bdf9f"}, + {file = "fonttools-4.55.2.tar.gz", hash = "sha256:45947e7b3f9673f91df125d375eb57b9a23f2a603f438a1aebf3171bffa7a205"}, ] [package.extras] @@ -2243,54 +2243,54 @@ files = [ [[package]] name = "memray" -version = "1.14.0" +version = "1.15.0" description = "A memory profiler for Python applications" optional = false python-versions = ">=3.7.0" files = [ - {file = "memray-1.14.0-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:745d9014cb662065501441a7b534c29914fe2b68398b37385aba9f4a1c51c723"}, - {file = "memray-1.14.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f62a402ca1a7126f749544c3d6493672d6330ffd37d59ba230bc73e5143b3bc2"}, - {file = "memray-1.14.0-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:36840f39277b1871ecb5a9592dd1aa517a17b9f855f4e3ff08aa328a9d305e69"}, - {file = "memray-1.14.0-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:3c7933ca70c0d59d0ce9b1064a6eda86231248759b46ed6dabedf489039d1aa1"}, - {file = "memray-1.14.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:75a5907345ff845652e709ddce3171a9ba2d65c62e8bd49a99131066e2a7ce3b"}, - {file = "memray-1.14.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:88c89c3797834eec177a89ad528699c75b94e2ed08c00754141eae69c520b894"}, - {file = "memray-1.14.0-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:d6087f291fd68acdf0a833efb57bc0f192c98ae89b4377c690c28313e78d029c"}, - {file = "memray-1.14.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e6ba7bff9dfa37bf3b80a5b83b50eadf20afb1f0e8de4a0139019154086d6bed"}, - {file = "memray-1.14.0-cp311-cp311-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:9bb0cfe1b755a860435cd52047b2e3f4f7b0c3887e0c1bf98da7127948284a91"}, - {file = "memray-1.14.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:638ba74e1735a40b6595fee7f37b426b9a95d244091a1f5df3dc5d98df1cbd4b"}, - {file = "memray-1.14.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7227ebf024cb0688a68ed91ed3e05c61a13751a9e875807195076b827bfde464"}, - {file = "memray-1.14.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:248dea8cfb5a615345e28b7e25c94377a8d198da3b6957ee443afa6f4ff1b733"}, - {file = "memray-1.14.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:7d03f6be66aa259df7fa50082876fbe6461108d77d46c1f720c46067d60685d4"}, - {file = "memray-1.14.0-cp312-cp312-macosx_10_14_x86_64.whl", hash = "sha256:9af9d30b1e484fd8591c9a7f322fd50b9192a2bce660be92385a01555af9968b"}, - {file = "memray-1.14.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:c4088b391c04796c888ac751b5d387f6e8212b3515d4c53ba540c65a6efe4bda"}, - {file = "memray-1.14.0-cp312-cp312-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:af8aee7e8e5cac1e4130f1184b3e03b6bb08264e4ba1696551791ed3f8fb824e"}, - {file = "memray-1.14.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4352f9e85957f2cbe45a9e1c87dfc19d2df77e93dcd8a558794a683eeee57b7b"}, - {file = "memray-1.14.0-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5953f0d2aa31b23d4cce20236a03d90b7c71903709a57b456d6494bfe6f470b7"}, - {file = "memray-1.14.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e4ccaca04365efcda51036fe2add980030e33cfc4f3a194a01f530a5c923c65"}, - {file = "memray-1.14.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:f85a27eb8a65c810161bb992116a66d328546f78a4a4c7c1868949651b917c08"}, - {file = "memray-1.14.0-cp313-cp313-macosx_10_14_x86_64.whl", hash = "sha256:958d57f7149b8fa4831785394f2a7ace93dbc2be6c49a1c07987a8972986474a"}, - {file = "memray-1.14.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:287a01953bc44dd0a32549a23bdacb5f9734e345ca289fa3923867c637715056"}, - {file = "memray-1.14.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dfc17cba35d98e3d2ca20ab995f17eec3edba7138b062cbc1aa36d53d9d2d955"}, - {file = "memray-1.14.0-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c82342cead930ca50235f59740ca238808f9c33ef31d994712972966beb6226e"}, - {file = "memray-1.14.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a22a826b4047e839310514f4889c24e45a66ea222fca19ac0ae7b2f89bbb0281"}, - {file = "memray-1.14.0-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:344f3c73b97ffc8f1666b404deafbc31a19e6b2881341b706aa7ec20afb0e8b1"}, - {file = "memray-1.14.0-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:a43455233d534e9c0e8dabe827d451124874a6455b2afcbcd60b823241ea5843"}, - {file = "memray-1.14.0-cp37-cp37m-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:e05a3b6bc82ef01821beaee98e86bd8de2ada06cb8781add9c40a3ae4a040383"}, - {file = "memray-1.14.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3bc27e5483c70236c9379b99277b4ea8fa4b3f73a99e37af81190649bd877881"}, - {file = "memray-1.14.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:6a7e5604448b2a78e329addfb099384515d3f973a03711c4e2a7b6c9f7f34f53"}, - {file = "memray-1.14.0-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:443885a96ab9f67d46288240e2593b5c3ecb2c507ddb4e3b10695e104403d001"}, - {file = "memray-1.14.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:52a45d96ed717d8efb645e99646a92dd21a2ca38bdb823fe22e38c429cba9513"}, - {file = "memray-1.14.0-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:72febec7b287667e8ea9ee3e879a4da19a4318bc47e211da815be74acd961994"}, - {file = "memray-1.14.0-cp38-cp38-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:4e07bdc3a4979b335c2b6b93a81b807d5aacd8dbbea56c41c6899a8bc0d2beb3"}, - {file = "memray-1.14.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3b5e729d03caf426dc45a258270537a603794ecc067ccfd92f9c67ba9332e788"}, - {file = "memray-1.14.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:1d0a1397b5387b75dc9d9759beb022cb360948584840e850474d7d39ad267f85"}, - {file = "memray-1.14.0-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:c119b600e7c665e0713f09e25f9ee09733a98035688ecc1ec8fd168fa37a77f6"}, - {file = "memray-1.14.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:29a2e7d84d1652ef4664bcceb155630979b4797180b70da35525d963a4ca707f"}, - {file = "memray-1.14.0-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:b3b8d46b6447cdecba3ba100d47c62e78cdad58b00b2d6ba004d6bad318c8668"}, - {file = "memray-1.14.0-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:57f9bf3f1c648f1ea877a84c21c449fdafd8cc105763ada6023e36bae9b45eb8"}, - {file = "memray-1.14.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5b7a59346d242fc39041d87a71cb6cf45baf492ffbb69da9690de49346be64a8"}, - {file = "memray-1.14.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:11fb00105572b70f2aca8b787ce9748b0c94672fbb6334f1604f7f813ca3dca6"}, - {file = "memray-1.14.0.tar.gz", hash = "sha256:b5d8874b7b215551f0ae9fa8aef3f2f52321a6460dc0141aaf9374709e6b0eb7"}, + {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"}, + {file = "memray-1.15.0-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:a5c6be5f9c2280b5ba077cbfec4706f209f9c0c2cd3a53d949ab9f4ee1f6a255"}, + {file = "memray-1.15.0-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:68bdad519b644539440914e1f6a04995631d0e31311ebe0977d949f2125bb579"}, + {file = "memray-1.15.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b4964c6bd555a0f1755dfdb97a8d9864e646054594449c66757441f7d7682405"}, + {file = "memray-1.15.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:92212b85c7d843126e4d343c8ca024f4a57537017b9ac7611864963b322aafae"}, + {file = "memray-1.15.0-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:cb8997e113378b9ac8bbd9b17f4f867fc5c1eea1980d873be3ebe4c2f1176784"}, + {file = "memray-1.15.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:8ee45d919d81bfeb33677357dd5d248f3cad1d56be2ebd1853d4615a9f965b11"}, + {file = "memray-1.15.0-cp311-cp311-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:a6b740aad69e7e5f82ffff53a8edef1313ff0b5e9b7253912da16e905dcb1dcb"}, + {file = "memray-1.15.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0045611f2da496e35d37a5ddfa2b6a74bbc82e47087924c07b3f520448297b26"}, + {file = "memray-1.15.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca5688e33a833de604d0e2de01b5bf11a4ac1d768998f8831a375a343dc7acaf"}, + {file = "memray-1.15.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4bbad938c3fdcebe0cf3c568fb8f8633ab37ab08ad4db167e0991e214d6f595b"}, + {file = "memray-1.15.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:f4eb50295bd87a091a85ec71f0ee612c5d709df490fea8a3adc4410f5da4f695"}, + {file = "memray-1.15.0-cp312-cp312-macosx_10_14_x86_64.whl", hash = "sha256:d13554a25129593872b5fbcd55ac34453239e51d9b6ace258329596ccce22bb3"}, + {file = "memray-1.15.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8cfe15962a9002ede8b1f8b4f045d95855100a8a60a9bf0d9f2b92950f914189"}, + {file = "memray-1.15.0-cp312-cp312-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:e84b39adca05e720bdbf950cc92ef4bafefa2d6160111e5fc427cf59c6c16d1a"}, + {file = "memray-1.15.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7745d2c58dfc33ef77f8827053cb957131420051b67e2d5642b605d0e65a586"}, + {file = "memray-1.15.0-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:412225d85db0ec22142a82646d85ecc1e8680d33adbfd15789c7eaa356ad4107"}, + {file = "memray-1.15.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d25ab7a7e32fedab46219121dfb6ec3e42c66984b217572fdd4cddc37359c521"}, + {file = "memray-1.15.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:fb885f92833279d34addc607831352e91267b8e547ea861ad561a3dba64f6757"}, + {file = "memray-1.15.0-cp313-cp313-macosx_10_14_x86_64.whl", hash = "sha256:c1308e6a5fc5bc4e183bc0fdf5e241ddd9fb374338f32d77a4d5e74ccf611ef1"}, + {file = "memray-1.15.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0794227dfa4b86a56137211fd5b8ec131e0bc4a5dc41c2f5a318ca56a22c9331"}, + {file = "memray-1.15.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f184e82debd4f0c8ecf8e6034efddccdd9fac22909553a7f094eabf0902cd53f"}, + {file = "memray-1.15.0-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3493c5ac1ae1353fd0d24481bc9f30da8960ef703bf4af966cefff9dd1234d38"}, + {file = "memray-1.15.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:145a3062d8bf631aa8dc4b0928585e201282634afc369799dae1a0b9ece59fd4"}, + {file = "memray-1.15.0-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:59a4ade09cfe46e85cdb3a1976e9768e4674a6e448533c415dbe84e5a834f7c3"}, + {file = "memray-1.15.0-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:bb9870f41fe0c4cd4612fded51174f5b837f3bc6364c57b4a60e65016ccc1f7a"}, + {file = "memray-1.15.0-cp37-cp37m-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:26cb3cac3810bbe9e701d40463c185cf9e7faac3965a0e2b309df1a9fc18cd9a"}, + {file = "memray-1.15.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:317287025cabd541f9fdec615b3c7ff394798113feea0edb92d31bc9f06eafd0"}, + {file = "memray-1.15.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:850eba1e3063d97172b0990a14f61782682baeb48f6ae039c0bb86b2f4d19b75"}, + {file = "memray-1.15.0-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:aa5150e3b58ba6184fac2a97426ee66f996dffe0571bbf09bffe23836318772e"}, + {file = "memray-1.15.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:753632eed43161131bb632799dc53b7ccb7e6341b8ca8ef4ad68ff8da81e766a"}, + {file = "memray-1.15.0-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:727190a81516e1955932c307ac6a55a3aedb5799bc2edf6a8fbf49852e851f0c"}, + {file = "memray-1.15.0-cp38-cp38-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:413b145445110900a99fb78b1fb6932c2e3ffadd35df5b258f8ac0a25e0aaf90"}, + {file = "memray-1.15.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2518a298ffa6c5a2ddfa6a36d196aa4aef5bb33c5d95a26565aac6a7f5fcb0c0"}, + {file = "memray-1.15.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:ae46cb726c4c06121614995b877365680f196fa4549698aa5026c494a40e1a24"}, + {file = "memray-1.15.0-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:ce28c6a4d89349c43d76ad35ff1c21057230086cfcf18c6f4c2305df108bf0cd"}, + {file = "memray-1.15.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:671c2fd8c835caad80c2023baf6cdc4326c0f6dd4ae8bf1d7dbf6ad700c13625"}, + {file = "memray-1.15.0-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:8404f3969e071e35364fd99d238da8ef245cf7ee2c790f3d46cd5b41cbac0541"}, + {file = "memray-1.15.0-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:a3e4c940deae29ea64d8dd4ffaee804f541a413c3c3c061a469837ed35d486b7"}, + {file = "memray-1.15.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:36720d9ee97dee6cd51b230cbd2556cc3e0215c5a569b97c1faebc927ac3c505"}, + {file = "memray-1.15.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:cba7727bfdee596f71323195af0262508ed0aec7ebbf67d98de0b959d9b8cf02"}, + {file = "memray-1.15.0.tar.gz", hash = "sha256:1beffa2bcba3dbe0f095d547927286eca46e272798b83026dd1b5db58e16ed56"}, ] [package.dependencies] @@ -2300,7 +2300,7 @@ textual = ">=0.41.0" [package.extras] benchmark = ["asv"] -dev = ["Cython", "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", "ipython", "isort", "mypy", "packaging", "pytest", "pytest-cov", "pytest-textual-snapshot", "setuptools", "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)"] @@ -3215,13 +3215,13 @@ files = [ [[package]] name = "pydantic" -version = "2.10.2" +version = "2.10.3" description = "Data validation using Python type hints" optional = false python-versions = ">=3.8" files = [ - {file = "pydantic-2.10.2-py3-none-any.whl", hash = "sha256:cfb96e45951117c3024e6b67b25cdc33a3cb7b2fa62e239f7af1378358a1d99e"}, - {file = "pydantic-2.10.2.tar.gz", hash = "sha256:2bc2d7f17232e0841cbba4641e65ba1eb6fafb3a08de3a091ff3ce14a197c4fa"}, + {file = "pydantic-2.10.3-py3-none-any.whl", hash = "sha256:be04d85bbc7b65651c5f8e6b9976ed9c6f41782a55524cef079a34a0bb82144d"}, + {file = "pydantic-2.10.3.tar.gz", hash = "sha256:cb5ac360ce894ceacd69c403187900a02c4b20b693a9dd1d643e1effab9eadf9"}, ] [package.dependencies] @@ -4262,6 +4262,11 @@ files = [ {file = "scikit_learn-1.5.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f60021ec1574e56632be2a36b946f8143bf4e5e6af4a06d85281adc22938e0dd"}, {file = "scikit_learn-1.5.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:394397841449853c2290a32050382edaec3da89e35b3e03d6cc966aebc6a8ae6"}, {file = "scikit_learn-1.5.2-cp312-cp312-win_amd64.whl", hash = "sha256:57cc1786cfd6bd118220a92ede80270132aa353647684efa385a74244a41e3b1"}, + {file = "scikit_learn-1.5.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:e9a702e2de732bbb20d3bad29ebd77fc05a6b427dc49964300340e4c9328b3f5"}, + {file = "scikit_learn-1.5.2-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:b0768ad641981f5d3a198430a1d31c3e044ed2e8a6f22166b4d546a5116d7908"}, + {file = "scikit_learn-1.5.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:178ddd0a5cb0044464fc1bfc4cca5b1833bfc7bb022d70b05db8530da4bb3dd3"}, + {file = "scikit_learn-1.5.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7284ade780084d94505632241bf78c44ab3b6f1e8ccab3d2af58e0e950f9c12"}, + {file = "scikit_learn-1.5.2-cp313-cp313-win_amd64.whl", hash = "sha256:b7b0f9a0b1040830d38c39b91b3a44e1b643f4b36e36567b80b7c6bd2202a27f"}, {file = "scikit_learn-1.5.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:757c7d514ddb00ae249832fe87100d9c73c6ea91423802872d9e74970a0e40b9"}, {file = "scikit_learn-1.5.2-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:52788f48b5d8bca5c0736c175fa6bdaab2ef00a8f536cda698db61bd89c551c1"}, {file = "scikit_learn-1.5.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:643964678f4b5fbdc95cbf8aec638acc7aa70f5f79ee2cdad1eec3df4ba6ead8"}, @@ -4411,13 +4416,13 @@ type = ["importlib_metadata (>=7.0.2)", "jaraco.develop (>=7.21)", "mypy (>=1.12 [[package]] name = "six" -version = "1.16.0" +version = "1.17.0" description = "Python 2 and 3 compatibility utilities" optional = false -python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*" +python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" files = [ - {file = "six-1.16.0-py2.py3-none-any.whl", hash = "sha256:8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254"}, - {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"}, + {file = "six-1.17.0-py2.py3-none-any.whl", hash = "sha256:4721f391ed90541fddacab5acf947aa0d3dc7d27b2e1e8eda2be8970586c3274"}, + {file = "six-1.17.0.tar.gz", hash = "sha256:ff70335d468e7eb6ec65b95b99d3a2836546063f63acc5171de367e834932a81"}, ] [[package]] @@ -4718,13 +4723,13 @@ test = ["pytest", "tornado (>=4.5)", "typeguard"] [[package]] name = "textual" -version = "0.88.1" +version = "0.89.1" description = "Modern Text User Interface framework" optional = false python-versions = "<4.0.0,>=3.8.1" files = [ - {file = "textual-0.88.1-py3-none-any.whl", hash = "sha256:f2db8ce892007f724dab57a2b791e55f9d7ce04d333c50fb4b6fb7f3990d4cec"}, - {file = "textual-0.88.1.tar.gz", hash = "sha256:9c56d953dc7d1a8ddf06acc910d9224027e02416551f92920e70f435bd28e062"}, + {file = "textual-0.89.1-py3-none-any.whl", hash = "sha256:0a5d214df6e951b4a2c421e13d0b608482882471c1e34ea74a3631adede8054f"}, + {file = "textual-0.89.1.tar.gz", hash = "sha256:66befe80e2bca5a8c876cd8ceeaf01752267b6b1dc1d0f73071f1f1e15d90cc8"}, ] [package.dependencies] @@ -4734,7 +4739,7 @@ rich = ">=13.3.3" typing-extensions = ">=4.4.0,<5.0.0" [package.extras] -syntax = ["tree-sitter (>=0.20.1,<0.21.0)", "tree-sitter-languages (==1.10.2)"] +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)"] [[package]] name = "threadpoolctl" @@ -4867,44 +4872,15 @@ files = [ {file = "types_cachetools-5.5.0.20240820-py3-none-any.whl", hash = "sha256:efb2ed8bf27a4b9d3ed70d33849f536362603a90b8090a328acf0cd42fda82e2"}, ] -[[package]] -name = "types-cffi" -version = "1.16.0.20240331" -description = "Typing stubs for cffi" -optional = false -python-versions = ">=3.8" -files = [ - {file = "types-cffi-1.16.0.20240331.tar.gz", hash = "sha256:b8b20d23a2b89cfed5f8c5bc53b0cb8677c3aac6d970dbc771e28b9c698f5dee"}, - {file = "types_cffi-1.16.0.20240331-py3-none-any.whl", hash = "sha256:a363e5ea54a4eb6a4a105d800685fde596bc318089b025b27dee09849fe41ff0"}, -] - -[package.dependencies] -types-setuptools = "*" - -[[package]] -name = "types-pyopenssl" -version = "24.1.0.20240722" -description = "Typing stubs for pyOpenSSL" -optional = false -python-versions = ">=3.8" -files = [ - {file = "types-pyOpenSSL-24.1.0.20240722.tar.gz", hash = "sha256:47913b4678a01d879f503a12044468221ed8576263c1540dcb0484ca21b08c39"}, - {file = "types_pyOpenSSL-24.1.0.20240722-py3-none-any.whl", hash = "sha256:6a7a5d2ec042537934cfb4c9d4deb0e16c4c6250b09358df1f083682fe6fda54"}, -] - -[package.dependencies] -cryptography = ">=35.0.0" -types-cffi = "*" - [[package]] name = "types-python-dateutil" -version = "2.9.0.20241003" +version = "2.9.0.20241206" description = "Typing stubs for python-dateutil" optional = false python-versions = ">=3.8" files = [ - {file = "types-python-dateutil-2.9.0.20241003.tar.gz", hash = "sha256:58cb85449b2a56d6684e41aeefb4c4280631246a0da1a719bdbe6f3fb0317446"}, - {file = "types_python_dateutil-2.9.0.20241003-py3-none-any.whl", hash = "sha256:250e1d8e80e7bbc3a6c99b907762711d1a1cdd00e978ad39cb5940f6f0a87f3d"}, + {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"}, ] [[package]] @@ -4929,21 +4905,6 @@ files = [ {file = "types_PyYAML-6.0.12.20240917-py3-none-any.whl", hash = "sha256:392b267f1c0fe6022952462bf5d6523f31e37f6cea49b14cee7ad634b6301570"}, ] -[[package]] -name = "types-redis" -version = "4.6.0.20241004" -description = "Typing stubs for redis" -optional = false -python-versions = ">=3.8" -files = [ - {file = "types-redis-4.6.0.20241004.tar.gz", hash = "sha256:5f17d2b3f9091ab75384153bfa276619ffa1cf6a38da60e10d5e6749cc5b902e"}, - {file = "types_redis-4.6.0.20241004-py3-none-any.whl", hash = "sha256:ef5da68cb827e5f606c8f9c0b49eeee4c2669d6d97122f301d3a55dc6a63f6ed"}, -] - -[package.dependencies] -cryptography = ">=35.0.0" -types-pyOpenSSL = "*" - [[package]] name = "types-requests" version = "2.32.0.20241016" @@ -4958,28 +4919,6 @@ files = [ [package.dependencies] urllib3 = ">=2" -[[package]] -name = "types-setuptools" -version = "75.6.0.20241126" -description = "Typing stubs for setuptools" -optional = false -python-versions = ">=3.8" -files = [ - {file = "types_setuptools-75.6.0.20241126-py3-none-any.whl", hash = "sha256:aaae310a0e27033c1da8457d4d26ac673b0c8a0de7272d6d4708e263f2ea3b9b"}, - {file = "types_setuptools-75.6.0.20241126.tar.gz", hash = "sha256:7bf25ad4be39740e469f9268b6beddda6e088891fa5a27e985c6ce68bf62ace0"}, -] - -[[package]] -name = "types-ujson" -version = "5.10.0.20240515" -description = "Typing stubs for ujson" -optional = false -python-versions = ">=3.8" -files = [ - {file = "types-ujson-5.10.0.20240515.tar.gz", hash = "sha256:ceae7127f0dafe4af5dd0ecf98ee13e9d75951ef963b5c5a9b7ea92e0d71f0d7"}, - {file = "types_ujson-5.10.0.20240515-py3-none-any.whl", hash = "sha256:02bafc36b3a93d2511757a64ff88bd505e0a57fba08183a9150fbcfcb2015310"}, -] - [[package]] name = "typing-extensions" version = "4.12.2" @@ -5334,4 +5273,4 @@ vector-db-based = ["cohere", "langchain", "openai", "tiktoken"] [metadata] lock-version = "2.0" python-versions = "^3.10,<3.13" -content-hash = "e0ef863b4ed5381ffe0ae40e03c327394611d3cbde70fc81e027aaa1d1db3f27" +content-hash = "74e1ddf0a21404cca651b5616ce00f77141cb3df15ee152785f3c2179ec9ae92" From c59ed5ab4218be688ab309e9d5093b324130707b Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Fri, 6 Dec 2024 15:58:31 +0200 Subject: [PATCH 08/34] Update poetry lock again --- poetry.lock | 288 ++++++++++++++++++++++++++-------------------------- 1 file changed, 144 insertions(+), 144 deletions(-) diff --git a/poetry.lock b/poetry.lock index bca96d5c7..d51debd86 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.8.4 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.8.3 and should not be changed by hand. [[package]] name = "aiohappyeyeballs" @@ -13,87 +13,87 @@ files = [ [[package]] name = "aiohttp" -version = "3.11.9" +version = "3.11.10" description = "Async http client/server framework (asyncio)" optional = true python-versions = ">=3.9" files = [ - {file = "aiohttp-3.11.9-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:0411777249f25d11bd2964a230b3ffafcbed6cd65d0f2b132bc2b8f5b8c347c7"}, - {file = "aiohttp-3.11.9-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:499368eb904566fbdf1a3836a1532000ef1308f34a1bcbf36e6351904cced771"}, - {file = "aiohttp-3.11.9-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0b5a5009b0159a8f707879dc102b139466d8ec6db05103ec1520394fdd8ea02c"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:176f8bb8931da0613bb0ed16326d01330066bb1e172dd97e1e02b1c27383277b"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6435a66957cdba1a0b16f368bde03ce9c79c57306b39510da6ae5312a1a5b2c1"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:202f40fb686e5f93908eee0c75d1e6fbe50a43e9bd4909bf3bf4a56b560ca180"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:39625703540feb50b6b7f938b3856d1f4886d2e585d88274e62b1bd273fae09b"}, - {file = "aiohttp-3.11.9-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c6beeac698671baa558e82fa160be9761cf0eb25861943f4689ecf9000f8ebd0"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:96726839a42429318017e67a42cca75d4f0d5248a809b3cc2e125445edd7d50d"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:3f5461c77649358610fb9694e790956b4238ac5d9e697a17f63619c096469afe"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:4313f3bc901255b22f01663eeeae167468264fdae0d32c25fc631d5d6e15b502"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:d6e274661c74195708fc4380a4ef64298926c5a50bb10fbae3d01627d7a075b7"}, - {file = "aiohttp-3.11.9-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:db2914de2559809fdbcf3e48f41b17a493b58cb7988d3e211f6b63126c55fe82"}, - {file = "aiohttp-3.11.9-cp310-cp310-win32.whl", hash = "sha256:27935716f8d62c1c73010428db310fd10136002cfc6d52b0ba7bdfa752d26066"}, - {file = "aiohttp-3.11.9-cp310-cp310-win_amd64.whl", hash = "sha256:afbe85b50ade42ddff5669947afde9e8a610e64d2c80be046d67ec4368e555fa"}, - {file = "aiohttp-3.11.9-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:afcda759a69c6a8be3aae764ec6733155aa4a5ad9aad4f398b52ba4037942fe3"}, - {file = "aiohttp-3.11.9-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c5bba6b83fde4ca233cfda04cbd4685ab88696b0c8eaf76f7148969eab5e248a"}, - {file = "aiohttp-3.11.9-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:442356e8924fe1a121f8c87866b0ecdc785757fd28924b17c20493961b3d6697"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f737fef6e117856400afee4f17774cdea392b28ecf058833f5eca368a18cf1bf"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ea142255d4901b03f89cb6a94411ecec117786a76fc9ab043af8f51dd50b5313"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6e1e9e447856e9b7b3d38e1316ae9a8c92e7536ef48373de758ea055edfd5db5"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e7f6173302f8a329ca5d1ee592af9e628d3ade87816e9958dcf7cdae2841def7"}, - {file = "aiohttp-3.11.9-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a7c6147c6306f537cff59409609508a1d2eff81199f0302dd456bb9e7ea50c39"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:e9d036a9a41fc78e8a3f10a86c2fc1098fca8fab8715ba9eb999ce4788d35df0"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:2ac9fd83096df36728da8e2f4488ac3b5602238f602706606f3702f07a13a409"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:d3108f0ad5c6b6d78eec5273219a5bbd884b4aacec17883ceefaac988850ce6e"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:96bbec47beb131bbf4bae05d8ef99ad9e5738f12717cfbbf16648b78b0232e87"}, - {file = "aiohttp-3.11.9-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:fc726c3fa8f606d07bd2b500e5dc4c0fd664c59be7788a16b9e34352c50b6b6b"}, - {file = "aiohttp-3.11.9-cp311-cp311-win32.whl", hash = "sha256:5720ebbc7a1b46c33a42d489d25d36c64c419f52159485e55589fbec648ea49a"}, - {file = "aiohttp-3.11.9-cp311-cp311-win_amd64.whl", hash = "sha256:17af09d963fa1acd7e4c280e9354aeafd9e3d47eaa4a6bfbd2171ad7da49f0c5"}, - {file = "aiohttp-3.11.9-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:c1f2d7fd583fc79c240094b3e7237d88493814d4b300d013a42726c35a734bc9"}, - {file = "aiohttp-3.11.9-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:d4b8a1b6c7a68c73191f2ebd3bf66f7ce02f9c374e309bdb68ba886bbbf1b938"}, - {file = "aiohttp-3.11.9-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:bd3f711f4c99da0091ced41dccdc1bcf8be0281dc314d6d9c6b6cf5df66f37a9"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:44cb1a1326a0264480a789e6100dc3e07122eb8cd1ad6b784a3d47d13ed1d89c"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7a7ddf981a0b953ade1c2379052d47ccda2f58ab678fca0671c7c7ca2f67aac2"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6ffa45cc55b18d4ac1396d1ddb029f139b1d3480f1594130e62bceadf2e1a838"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cca505829cdab58c2495ff418c96092d225a1bbd486f79017f6de915580d3c44"}, - {file = "aiohttp-3.11.9-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:44d323aa80a867cb6db6bebb4bbec677c6478e38128847f2c6b0f70eae984d72"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:b2fab23003c4bb2249729a7290a76c1dda38c438300fdf97d4e42bf78b19c810"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:be0c7c98e38a1e3ad7a6ff64af8b6d6db34bf5a41b1478e24c3c74d9e7f8ed42"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:5cc5e0d069c56645446c45a4b5010d4b33ac6c5ebfd369a791b5f097e46a3c08"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9bcf97b971289be69638d8b1b616f7e557e1342debc7fc86cf89d3f08960e411"}, - {file = "aiohttp-3.11.9-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c7333e7239415076d1418dbfb7fa4df48f3a5b00f8fdf854fca549080455bc14"}, - {file = "aiohttp-3.11.9-cp312-cp312-win32.whl", hash = "sha256:9384b07cfd3045b37b05ed002d1c255db02fb96506ad65f0f9b776b762a7572e"}, - {file = "aiohttp-3.11.9-cp312-cp312-win_amd64.whl", hash = "sha256:f5252ba8b43906f206048fa569debf2cd0da0316e8d5b4d25abe53307f573941"}, - {file = "aiohttp-3.11.9-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:282e0a7ddd36ebc411f156aeaa0491e8fe7f030e2a95da532cf0c84b0b70bc66"}, - {file = "aiohttp-3.11.9-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:ebd3e6b0c7d4954cca59d241970011f8d3327633d555051c430bd09ff49dc494"}, - {file = "aiohttp-3.11.9-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:30f9f89ae625d412043f12ca3771b2ccec227cc93b93bb1f994db6e1af40a7d3"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7a3b5b2c012d70c63d9d13c57ed1603709a4d9d7d473e4a9dfece0e4ea3d5f51"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6ef1550bb5f55f71b97a6a395286db07f7f2c01c8890e613556df9a51da91e8d"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:317251b9c9a2f1a9ff9cd093775b34c6861d1d7df9439ce3d32a88c275c995cd"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21cbe97839b009826a61b143d3ca4964c8590d7aed33d6118125e5b71691ca46"}, - {file = "aiohttp-3.11.9-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:618b18c3a2360ac940a5503da14fa4f880c5b9bc315ec20a830357bcc62e6bae"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a0cf4d814689e58f57ecd5d8c523e6538417ca2e72ff52c007c64065cef50fb2"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:15c4e489942d987d5dac0ba39e5772dcbed4cc9ae3710d1025d5ba95e4a5349c"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:ec8df0ff5a911c6d21957a9182402aad7bf060eaeffd77c9ea1c16aecab5adbf"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:ed95d66745f53e129e935ad726167d3a6cb18c5d33df3165974d54742c373868"}, - {file = "aiohttp-3.11.9-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:647ec5bee7e4ec9f1034ab48173b5fa970d9a991e565549b965e93331f1328fe"}, - {file = "aiohttp-3.11.9-cp313-cp313-win32.whl", hash = "sha256:ef2c9499b7bd1e24e473dc1a85de55d72fd084eea3d8bdeec7ee0720decb54fa"}, - {file = "aiohttp-3.11.9-cp313-cp313-win_amd64.whl", hash = "sha256:84de955314aa5e8d469b00b14d6d714b008087a0222b0f743e7ffac34ef56aff"}, - {file = "aiohttp-3.11.9-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:e738aabff3586091221044b7a584865ddc4d6120346d12e28e788307cd731043"}, - {file = "aiohttp-3.11.9-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:28f29bce89c3b401a53d6fd4bee401ee943083bf2bdc12ef297c1d63155070b0"}, - {file = "aiohttp-3.11.9-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:31de2f10f63f96cc19e04bd2df9549559beadd0b2ee2da24a17e7ed877ca8c60"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77f31cebd8c27a36af6c7346055ac564946e562080ee1a838da724585c67474f"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0bcb7f6976dc0b6b56efde13294862adf68dd48854111b422a336fa729a82ea6"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1a8b13b9950d8b2f8f58b6e5842c4b842b5887e2c32e3f4644d6642f1659a530"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c9c23e62f3545c2216100603614f9e019e41b9403c47dd85b8e7e5015bf1bde0"}, - {file = "aiohttp-3.11.9-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ec656680fc53a13f849c71afd0c84a55c536206d524cbc831cde80abbe80489e"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:36df00e0541f264ce42d62280281541a47474dfda500bc5b7f24f70a7f87be7a"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:8dcfd14c712aa9dd18049280bfb2f95700ff6a8bde645e09f17c3ed3f05a0130"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:14624d96f0d69cf451deed3173079a68c322279be6030208b045ab77e1e8d550"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:4b01d9cfcb616eeb6d40f02e66bebfe7b06d9f2ef81641fdd50b8dd981166e0b"}, - {file = "aiohttp-3.11.9-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:928f92f80e2e8d6567b87d3316c1fd9860ccfe36e87a9a7f5237d4cda8baa1ba"}, - {file = "aiohttp-3.11.9-cp39-cp39-win32.whl", hash = "sha256:c8a02f74ae419e3955af60f570d83187423e42e672a6433c5e292f1d23619269"}, - {file = "aiohttp-3.11.9-cp39-cp39-win_amd64.whl", hash = "sha256:0a97d657f6cf8782a830bb476c13f7d777cfcab8428ac49dde15c22babceb361"}, - {file = "aiohttp-3.11.9.tar.gz", hash = "sha256:a9266644064779840feec0e34f10a89b3ff1d2d6b751fe90017abcad1864fa7c"}, + {file = "aiohttp-3.11.10-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cbad88a61fa743c5d283ad501b01c153820734118b65aee2bd7dbb735475ce0d"}, + {file = "aiohttp-3.11.10-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:80886dac673ceaef499de2f393fc80bb4481a129e6cb29e624a12e3296cc088f"}, + {file = "aiohttp-3.11.10-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:61b9bae80ed1f338c42f57c16918853dc51775fb5cb61da70d590de14d8b5fb4"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9e2e576caec5c6a6b93f41626c9c02fc87cd91538b81a3670b2e04452a63def6"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:02c13415b5732fb6ee7ff64583a5e6ed1c57aa68f17d2bda79c04888dfdc2769"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4cfce37f31f20800a6a6620ce2cdd6737b82e42e06e6e9bd1b36f546feb3c44f"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3bbbfff4c679c64e6e23cb213f57cc2c9165c9a65d63717108a644eb5a7398df"}, + {file = "aiohttp-3.11.10-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:49c7dbbc1a559ae14fc48387a115b7d4bbc84b4a2c3b9299c31696953c2a5219"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:68386d78743e6570f054fe7949d6cb37ef2b672b4d3405ce91fafa996f7d9b4d"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:9ef405356ba989fb57f84cac66f7b0260772836191ccefbb987f414bcd2979d9"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:5d6958671b296febe7f5f859bea581a21c1d05430d1bbdcf2b393599b1cdce77"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:99b7920e7165be5a9e9a3a7f1b680f06f68ff0d0328ff4079e5163990d046767"}, + {file = "aiohttp-3.11.10-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0dc49f42422163efb7e6f1df2636fe3db72713f6cd94688e339dbe33fe06d61d"}, + {file = "aiohttp-3.11.10-cp310-cp310-win32.whl", hash = "sha256:40d1c7a7f750b5648642586ba7206999650208dbe5afbcc5284bcec6579c9b91"}, + {file = "aiohttp-3.11.10-cp310-cp310-win_amd64.whl", hash = "sha256:68ff6f48b51bd78ea92b31079817aff539f6c8fc80b6b8d6ca347d7c02384e33"}, + {file = "aiohttp-3.11.10-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:77c4aa15a89847b9891abf97f3d4048f3c2d667e00f8a623c89ad2dccee6771b"}, + {file = "aiohttp-3.11.10-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:909af95a72cedbefe5596f0bdf3055740f96c1a4baa0dd11fd74ca4de0b4e3f1"}, + {file = "aiohttp-3.11.10-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:386fbe79863eb564e9f3615b959e28b222259da0c48fd1be5929ac838bc65683"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3de34936eb1a647aa919655ff8d38b618e9f6b7f250cc19a57a4bf7fd2062b6d"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0c9527819b29cd2b9f52033e7fb9ff08073df49b4799c89cb5754624ecd98299"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:65a96e3e03300b41f261bbfd40dfdbf1c301e87eab7cd61c054b1f2e7c89b9e8"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:98f5635f7b74bcd4f6f72fcd85bea2154b323a9f05226a80bc7398d0c90763b0"}, + {file = "aiohttp-3.11.10-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:03b6002e20938fc6ee0918c81d9e776bebccc84690e2b03ed132331cca065ee5"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:6362cc6c23c08d18ddbf0e8c4d5159b5df74fea1a5278ff4f2c79aed3f4e9f46"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:3691ed7726fef54e928fe26344d930c0c8575bc968c3e239c2e1a04bd8cf7838"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:31d5093d3acd02b31c649d3a69bb072d539d4c7659b87caa4f6d2bcf57c2fa2b"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:8b3cf2dc0f0690a33f2d2b2cb15db87a65f1c609f53c37e226f84edb08d10f52"}, + {file = "aiohttp-3.11.10-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:fbbaea811a2bba171197b08eea288b9402faa2bab2ba0858eecdd0a4105753a3"}, + {file = "aiohttp-3.11.10-cp311-cp311-win32.whl", hash = "sha256:4b2c7ac59c5698a7a8207ba72d9e9c15b0fc484a560be0788b31312c2c5504e4"}, + {file = "aiohttp-3.11.10-cp311-cp311-win_amd64.whl", hash = "sha256:974d3a2cce5fcfa32f06b13ccc8f20c6ad9c51802bb7f829eae8a1845c4019ec"}, + {file = "aiohttp-3.11.10-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:b78f053a7ecfc35f0451d961dacdc671f4bcbc2f58241a7c820e9d82559844cf"}, + {file = "aiohttp-3.11.10-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:ab7485222db0959a87fbe8125e233b5a6f01f4400785b36e8a7878170d8c3138"}, + {file = "aiohttp-3.11.10-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:cf14627232dfa8730453752e9cdc210966490992234d77ff90bc8dc0dce361d5"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:076bc454a7e6fd646bc82ea7f98296be0b1219b5e3ef8a488afbdd8e81fbac50"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:482cafb7dc886bebeb6c9ba7925e03591a62ab34298ee70d3dd47ba966370d2c"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bf3d1a519a324af764a46da4115bdbd566b3c73fb793ffb97f9111dbc684fc4d"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:24213ba85a419103e641e55c27dc7ff03536c4873470c2478cce3311ba1eee7b"}, + {file = "aiohttp-3.11.10-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b99acd4730ad1b196bfb03ee0803e4adac371ae8efa7e1cbc820200fc5ded109"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:14cdb5a9570be5a04eec2ace174a48ae85833c2aadc86de68f55541f66ce42ab"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:7e97d622cb083e86f18317282084bc9fbf261801b0192c34fe4b1febd9f7ae69"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:012f176945af138abc10c4a48743327a92b4ca9adc7a0e078077cdb5dbab7be0"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:44224d815853962f48fe124748227773acd9686eba6dc102578defd6fc99e8d9"}, + {file = "aiohttp-3.11.10-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c87bf31b7fdab94ae3adbe4a48e711bfc5f89d21cf4c197e75561def39e223bc"}, + {file = "aiohttp-3.11.10-cp312-cp312-win32.whl", hash = "sha256:06a8e2ee1cbac16fe61e51e0b0c269400e781b13bcfc33f5425912391a542985"}, + {file = "aiohttp-3.11.10-cp312-cp312-win_amd64.whl", hash = "sha256:be2b516f56ea883a3e14dda17059716593526e10fb6303189aaf5503937db408"}, + {file = "aiohttp-3.11.10-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:8cc5203b817b748adccb07f36390feb730b1bc5f56683445bfe924fc270b8816"}, + {file = "aiohttp-3.11.10-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:5ef359ebc6949e3a34c65ce20230fae70920714367c63afd80ea0c2702902ccf"}, + {file = "aiohttp-3.11.10-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:9bca390cb247dbfaec3c664326e034ef23882c3f3bfa5fbf0b56cad0320aaca5"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:811f23b3351ca532af598405db1093f018edf81368e689d1b508c57dcc6b6a32"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ddf5f7d877615f6a1e75971bfa5ac88609af3b74796ff3e06879e8422729fd01"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6ab29b8a0beb6f8eaf1e5049252cfe74adbaafd39ba91e10f18caeb0e99ffb34"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c49a76c1038c2dd116fa443eba26bbb8e6c37e924e2513574856de3b6516be99"}, + {file = "aiohttp-3.11.10-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7f3dc0e330575f5b134918976a645e79adf333c0a1439dcf6899a80776c9ab39"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:efb15a17a12497685304b2d976cb4939e55137df7b09fa53f1b6a023f01fcb4e"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:db1d0b28fcb7f1d35600150c3e4b490775251dea70f894bf15c678fdd84eda6a"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:15fccaf62a4889527539ecb86834084ecf6e9ea70588efde86e8bc775e0e7542"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:593c114a2221444f30749cc5e5f4012488f56bd14de2af44fe23e1e9894a9c60"}, + {file = "aiohttp-3.11.10-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:7852bbcb4d0d2f0c4d583f40c3bc750ee033265d80598d0f9cb6f372baa6b836"}, + {file = "aiohttp-3.11.10-cp313-cp313-win32.whl", hash = "sha256:65e55ca7debae8faaffee0ebb4b47a51b4075f01e9b641c31e554fd376595c6c"}, + {file = "aiohttp-3.11.10-cp313-cp313-win_amd64.whl", hash = "sha256:beb39a6d60a709ae3fb3516a1581777e7e8b76933bb88c8f4420d875bb0267c6"}, + {file = "aiohttp-3.11.10-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:0580f2e12de2138f34debcd5d88894786453a76e98febaf3e8fe5db62d01c9bf"}, + {file = "aiohttp-3.11.10-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a55d2ad345684e7c3dd2c20d2f9572e9e1d5446d57200ff630e6ede7612e307f"}, + {file = "aiohttp-3.11.10-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:04814571cb72d65a6899db6099e377ed00710bf2e3eafd2985166f2918beaf59"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e44a9a3c053b90c6f09b1bb4edd880959f5328cf63052503f892c41ea786d99f"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:502a1464ccbc800b4b1995b302efaf426e8763fadf185e933c2931df7db9a199"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:613e5169f8ae77b1933e42e418a95931fb4867b2991fc311430b15901ed67079"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4cca22a61b7fe45da8fc73c3443150c3608750bbe27641fc7558ec5117b27fdf"}, + {file = "aiohttp-3.11.10-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:86a5dfcc39309470bd7b68c591d84056d195428d5d2e0b5ccadfbaf25b026ebc"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:77ae58586930ee6b2b6f696c82cf8e78c8016ec4795c53e36718365f6959dc82"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:78153314f26d5abef3239b4a9af20c229c6f3ecb97d4c1c01b22c4f87669820c"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:98283b94cc0e11c73acaf1c9698dea80c830ca476492c0fe2622bd931f34b487"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:53bf2097e05c2accc166c142a2090e4c6fd86581bde3fd9b2d3f9e93dda66ac1"}, + {file = "aiohttp-3.11.10-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:c5532f0441fc09c119e1dca18fbc0687e64fbeb45aa4d6a87211ceaee50a74c4"}, + {file = "aiohttp-3.11.10-cp39-cp39-win32.whl", hash = "sha256:47ad15a65fb41c570cd0ad9a9ff8012489e68176e7207ec7b82a0940dddfd8be"}, + {file = "aiohttp-3.11.10-cp39-cp39-win_amd64.whl", hash = "sha256:c6b9e6d7e41656d78e37ce754813fa44b455c3d0d0dced2a047def7dc5570b74"}, + {file = "aiohttp-3.11.10.tar.gz", hash = "sha256:b1fc6b45010a8d0ff9e88f9f2418c6fd408c99c211257334aff41597ebece42e"}, ] [package.dependencies] @@ -649,73 +649,73 @@ test-no-images = ["pytest", "pytest-cov", "pytest-rerunfailures", "pytest-xdist" [[package]] name = "coverage" -version = "7.6.8" +version = "7.6.9" description = "Code coverage measurement for Python" optional = false python-versions = ">=3.9" files = [ - {file = "coverage-7.6.8-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b39e6011cd06822eb964d038d5dff5da5d98652b81f5ecd439277b32361a3a50"}, - {file = "coverage-7.6.8-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:63c19702db10ad79151a059d2d6336fe0c470f2e18d0d4d1a57f7f9713875dcf"}, - {file = "coverage-7.6.8-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3985b9be361d8fb6b2d1adc9924d01dec575a1d7453a14cccd73225cb79243ee"}, - {file = "coverage-7.6.8-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:644ec81edec0f4ad17d51c838a7d01e42811054543b76d4ba2c5d6af741ce2a6"}, - {file = "coverage-7.6.8-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1f188a2402f8359cf0c4b1fe89eea40dc13b52e7b4fd4812450da9fcd210181d"}, - {file = "coverage-7.6.8-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:e19122296822deafce89a0c5e8685704c067ae65d45e79718c92df7b3ec3d331"}, - {file = "coverage-7.6.8-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:13618bed0c38acc418896005732e565b317aa9e98d855a0e9f211a7ffc2d6638"}, - {file = "coverage-7.6.8-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:193e3bffca48ad74b8c764fb4492dd875038a2f9925530cb094db92bb5e47bed"}, - {file = "coverage-7.6.8-cp310-cp310-win32.whl", hash = "sha256:3988665ee376abce49613701336544041f2117de7b7fbfe91b93d8ff8b151c8e"}, - {file = "coverage-7.6.8-cp310-cp310-win_amd64.whl", hash = "sha256:f56f49b2553d7dd85fd86e029515a221e5c1f8cb3d9c38b470bc38bde7b8445a"}, - {file = "coverage-7.6.8-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:86cffe9c6dfcfe22e28027069725c7f57f4b868a3f86e81d1c62462764dc46d4"}, - {file = "coverage-7.6.8-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d82ab6816c3277dc962cfcdc85b1efa0e5f50fb2c449432deaf2398a2928ab94"}, - {file = "coverage-7.6.8-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:13690e923a3932e4fad4c0ebfb9cb5988e03d9dcb4c5150b5fcbf58fd8bddfc4"}, - {file = "coverage-7.6.8-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4be32da0c3827ac9132bb488d331cb32e8d9638dd41a0557c5569d57cf22c9c1"}, - {file = "coverage-7.6.8-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:44e6c85bbdc809383b509d732b06419fb4544dca29ebe18480379633623baafb"}, - {file = "coverage-7.6.8-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:768939f7c4353c0fac2f7c37897e10b1414b571fd85dd9fc49e6a87e37a2e0d8"}, - {file = "coverage-7.6.8-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e44961e36cb13c495806d4cac67640ac2866cb99044e210895b506c26ee63d3a"}, - {file = "coverage-7.6.8-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:3ea8bb1ab9558374c0ab591783808511d135a833c3ca64a18ec927f20c4030f0"}, - {file = "coverage-7.6.8-cp311-cp311-win32.whl", hash = "sha256:629a1ba2115dce8bf75a5cce9f2486ae483cb89c0145795603d6554bdc83e801"}, - {file = "coverage-7.6.8-cp311-cp311-win_amd64.whl", hash = "sha256:fb9fc32399dca861584d96eccd6c980b69bbcd7c228d06fb74fe53e007aa8ef9"}, - {file = "coverage-7.6.8-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:e683e6ecc587643f8cde8f5da6768e9d165cd31edf39ee90ed7034f9ca0eefee"}, - {file = "coverage-7.6.8-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1defe91d41ce1bd44b40fabf071e6a01a5aa14de4a31b986aa9dfd1b3e3e414a"}, - {file = "coverage-7.6.8-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7ad66e8e50225ebf4236368cc43c37f59d5e6728f15f6e258c8639fa0dd8e6d"}, - {file = "coverage-7.6.8-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3fe47da3e4fda5f1abb5709c156eca207eacf8007304ce3019eb001e7a7204cb"}, - {file = "coverage-7.6.8-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:202a2d645c5a46b84992f55b0a3affe4f0ba6b4c611abec32ee88358db4bb649"}, - {file = "coverage-7.6.8-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:4674f0daa1823c295845b6a740d98a840d7a1c11df00d1fd62614545c1583787"}, - {file = "coverage-7.6.8-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:74610105ebd6f33d7c10f8907afed696e79c59e3043c5f20eaa3a46fddf33b4c"}, - {file = "coverage-7.6.8-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:37cda8712145917105e07aab96388ae76e787270ec04bcb9d5cc786d7cbb8443"}, - {file = "coverage-7.6.8-cp312-cp312-win32.whl", hash = "sha256:9e89d5c8509fbd6c03d0dd1972925b22f50db0792ce06324ba069f10787429ad"}, - {file = "coverage-7.6.8-cp312-cp312-win_amd64.whl", hash = "sha256:379c111d3558272a2cae3d8e57e6b6e6f4fe652905692d54bad5ea0ca37c5ad4"}, - {file = "coverage-7.6.8-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:0b0c69f4f724c64dfbfe79f5dfb503b42fe6127b8d479b2677f2b227478db2eb"}, - {file = "coverage-7.6.8-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:c15b32a7aca8038ed7644f854bf17b663bc38e1671b5d6f43f9a2b2bd0c46f63"}, - {file = "coverage-7.6.8-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:63068a11171e4276f6ece913bde059e77c713b48c3a848814a6537f35afb8365"}, - {file = "coverage-7.6.8-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6f4548c5ead23ad13fb7a2c8ea541357474ec13c2b736feb02e19a3085fac002"}, - {file = "coverage-7.6.8-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b4b4299dd0d2c67caaaf286d58aef5e75b125b95615dda4542561a5a566a1e3"}, - {file = "coverage-7.6.8-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:c9ebfb2507751f7196995142f057d1324afdab56db1d9743aab7f50289abd022"}, - {file = "coverage-7.6.8-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:c1b4474beee02ede1eef86c25ad4600a424fe36cff01a6103cb4533c6bf0169e"}, - {file = "coverage-7.6.8-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:d9fd2547e6decdbf985d579cf3fc78e4c1d662b9b0ff7cc7862baaab71c9cc5b"}, - {file = "coverage-7.6.8-cp313-cp313-win32.whl", hash = "sha256:8aae5aea53cbfe024919715eca696b1a3201886ce83790537d1c3668459c7146"}, - {file = "coverage-7.6.8-cp313-cp313-win_amd64.whl", hash = "sha256:ae270e79f7e169ccfe23284ff5ea2d52a6f401dc01b337efb54b3783e2ce3f28"}, - {file = "coverage-7.6.8-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:de38add67a0af869b0d79c525d3e4588ac1ffa92f39116dbe0ed9753f26eba7d"}, - {file = "coverage-7.6.8-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:b07c25d52b1c16ce5de088046cd2432b30f9ad5e224ff17c8f496d9cb7d1d451"}, - {file = "coverage-7.6.8-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:62a66ff235e4c2e37ed3b6104d8b478d767ff73838d1222132a7a026aa548764"}, - {file = "coverage-7.6.8-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:09b9f848b28081e7b975a3626e9081574a7b9196cde26604540582da60235fdf"}, - {file = "coverage-7.6.8-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:093896e530c38c8e9c996901858ac63f3d4171268db2c9c8b373a228f459bbc5"}, - {file = "coverage-7.6.8-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:9a7b8ac36fd688c8361cbc7bf1cb5866977ece6e0b17c34aa0df58bda4fa18a4"}, - {file = "coverage-7.6.8-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:38c51297b35b3ed91670e1e4efb702b790002e3245a28c76e627478aa3c10d83"}, - {file = "coverage-7.6.8-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:2e4e0f60cb4bd7396108823548e82fdab72d4d8a65e58e2c19bbbc2f1e2bfa4b"}, - {file = "coverage-7.6.8-cp313-cp313t-win32.whl", hash = "sha256:6535d996f6537ecb298b4e287a855f37deaf64ff007162ec0afb9ab8ba3b8b71"}, - {file = "coverage-7.6.8-cp313-cp313t-win_amd64.whl", hash = "sha256:c79c0685f142ca53256722a384540832420dff4ab15fec1863d7e5bc8691bdcc"}, - {file = "coverage-7.6.8-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3ac47fa29d8d41059ea3df65bd3ade92f97ee4910ed638e87075b8e8ce69599e"}, - {file = "coverage-7.6.8-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:24eda3a24a38157eee639ca9afe45eefa8d2420d49468819ac5f88b10de84f4c"}, - {file = "coverage-7.6.8-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e4c81ed2820b9023a9a90717020315e63b17b18c274a332e3b6437d7ff70abe0"}, - {file = "coverage-7.6.8-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:bd55f8fc8fa494958772a2a7302b0354ab16e0b9272b3c3d83cdb5bec5bd1779"}, - {file = "coverage-7.6.8-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f39e2f3530ed1626c66e7493be7a8423b023ca852aacdc91fb30162c350d2a92"}, - {file = "coverage-7.6.8-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:716a78a342679cd1177bc8c2fe957e0ab91405bd43a17094324845200b2fddf4"}, - {file = "coverage-7.6.8-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:177f01eeaa3aee4a5ffb0d1439c5952b53d5010f86e9d2667963e632e30082cc"}, - {file = "coverage-7.6.8-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:912e95017ff51dc3d7b6e2be158dedc889d9a5cc3382445589ce554f1a34c0ea"}, - {file = "coverage-7.6.8-cp39-cp39-win32.whl", hash = "sha256:4db3ed6a907b555e57cc2e6f14dc3a4c2458cdad8919e40b5357ab9b6db6c43e"}, - {file = "coverage-7.6.8-cp39-cp39-win_amd64.whl", hash = "sha256:428ac484592f780e8cd7b6b14eb568f7c85460c92e2a37cb0c0e5186e1a0d076"}, - {file = "coverage-7.6.8-pp39.pp310-none-any.whl", hash = "sha256:5c52a036535d12590c32c49209e79cabaad9f9ad8aa4cbd875b68c4d67a9cbce"}, - {file = "coverage-7.6.8.tar.gz", hash = "sha256:8b2b8503edb06822c86d82fa64a4a5cb0760bb8f31f26e138ec743f422f37cfc"}, + {file = "coverage-7.6.9-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:85d9636f72e8991a1706b2b55b06c27545448baf9f6dbf51c4004609aacd7dcb"}, + {file = "coverage-7.6.9-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:608a7fd78c67bee8936378299a6cb9f5149bb80238c7a566fc3e6717a4e68710"}, + {file = "coverage-7.6.9-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:96d636c77af18b5cb664ddf12dab9b15a0cfe9c0bde715da38698c8cea748bfa"}, + {file = "coverage-7.6.9-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d75cded8a3cff93da9edc31446872d2997e327921d8eed86641efafd350e1df1"}, + {file = "coverage-7.6.9-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7b15f589593110ae767ce997775d645b47e5cbbf54fd322f8ebea6277466cec"}, + {file = "coverage-7.6.9-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:44349150f6811b44b25574839b39ae35291f6496eb795b7366fef3bd3cf112d3"}, + {file = "coverage-7.6.9-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:d891c136b5b310d0e702e186d70cd16d1119ea8927347045124cb286b29297e5"}, + {file = "coverage-7.6.9-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:db1dab894cc139f67822a92910466531de5ea6034ddfd2b11c0d4c6257168073"}, + {file = "coverage-7.6.9-cp310-cp310-win32.whl", hash = "sha256:41ff7b0da5af71a51b53f501a3bac65fb0ec311ebed1632e58fc6107f03b9198"}, + {file = "coverage-7.6.9-cp310-cp310-win_amd64.whl", hash = "sha256:35371f8438028fdccfaf3570b31d98e8d9eda8bb1d6ab9473f5a390969e98717"}, + {file = "coverage-7.6.9-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:932fc826442132dde42ee52cf66d941f581c685a6313feebed358411238f60f9"}, + {file = "coverage-7.6.9-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:085161be5f3b30fd9b3e7b9a8c301f935c8313dcf928a07b116324abea2c1c2c"}, + {file = "coverage-7.6.9-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ccc660a77e1c2bf24ddbce969af9447a9474790160cfb23de6be4fa88e3951c7"}, + {file = "coverage-7.6.9-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c69e42c892c018cd3c8d90da61d845f50a8243062b19d228189b0224150018a9"}, + {file = "coverage-7.6.9-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0824a28ec542a0be22f60c6ac36d679e0e262e5353203bea81d44ee81fe9c6d4"}, + {file = "coverage-7.6.9-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:4401ae5fc52ad8d26d2a5d8a7428b0f0c72431683f8e63e42e70606374c311a1"}, + {file = "coverage-7.6.9-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:98caba4476a6c8d59ec1eb00c7dd862ba9beca34085642d46ed503cc2d440d4b"}, + {file = "coverage-7.6.9-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ee5defd1733fd6ec08b168bd4f5387d5b322f45ca9e0e6c817ea6c4cd36313e3"}, + {file = "coverage-7.6.9-cp311-cp311-win32.whl", hash = "sha256:f2d1ec60d6d256bdf298cb86b78dd715980828f50c46701abc3b0a2b3f8a0dc0"}, + {file = "coverage-7.6.9-cp311-cp311-win_amd64.whl", hash = "sha256:0d59fd927b1f04de57a2ba0137166d31c1a6dd9e764ad4af552912d70428c92b"}, + {file = "coverage-7.6.9-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:99e266ae0b5d15f1ca8d278a668df6f51cc4b854513daab5cae695ed7b721cf8"}, + {file = "coverage-7.6.9-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:9901d36492009a0a9b94b20e52ebfc8453bf49bb2b27bca2c9706f8b4f5a554a"}, + {file = "coverage-7.6.9-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:abd3e72dd5b97e3af4246cdada7738ef0e608168de952b837b8dd7e90341f015"}, + {file = "coverage-7.6.9-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ff74026a461eb0660366fb01c650c1d00f833a086b336bdad7ab00cc952072b3"}, + {file = "coverage-7.6.9-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:65dad5a248823a4996724a88eb51d4b31587aa7aa428562dbe459c684e5787ae"}, + {file = "coverage-7.6.9-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:22be16571504c9ccea919fcedb459d5ab20d41172056206eb2994e2ff06118a4"}, + {file = "coverage-7.6.9-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f957943bc718b87144ecaee70762bc2bc3f1a7a53c7b861103546d3a403f0a6"}, + {file = "coverage-7.6.9-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:0ae1387db4aecb1f485fb70a6c0148c6cdaebb6038f1d40089b1fc84a5db556f"}, + {file = "coverage-7.6.9-cp312-cp312-win32.whl", hash = "sha256:1a330812d9cc7ac2182586f6d41b4d0fadf9be9049f350e0efb275c8ee8eb692"}, + {file = "coverage-7.6.9-cp312-cp312-win_amd64.whl", hash = "sha256:b12c6b18269ca471eedd41c1b6a1065b2f7827508edb9a7ed5555e9a56dcfc97"}, + {file = "coverage-7.6.9-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:899b8cd4781c400454f2f64f7776a5d87bbd7b3e7f7bda0cb18f857bb1334664"}, + {file = "coverage-7.6.9-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:61f70dc68bd36810972e55bbbe83674ea073dd1dcc121040a08cdf3416c5349c"}, + {file = "coverage-7.6.9-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8a289d23d4c46f1a82d5db4abeb40b9b5be91731ee19a379d15790e53031c014"}, + {file = "coverage-7.6.9-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7e216d8044a356fc0337c7a2a0536d6de07888d7bcda76febcb8adc50bdbbd00"}, + {file = "coverage-7.6.9-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3c026eb44f744acaa2bda7493dad903aa5bf5fc4f2554293a798d5606710055d"}, + {file = "coverage-7.6.9-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:e77363e8425325384f9d49272c54045bbed2f478e9dd698dbc65dbc37860eb0a"}, + {file = "coverage-7.6.9-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:777abfab476cf83b5177b84d7486497e034eb9eaea0d746ce0c1268c71652077"}, + {file = "coverage-7.6.9-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:447af20e25fdbe16f26e84eb714ba21d98868705cb138252d28bc400381f6ffb"}, + {file = "coverage-7.6.9-cp313-cp313-win32.whl", hash = "sha256:d872ec5aeb086cbea771c573600d47944eea2dcba8be5f3ee649bfe3cb8dc9ba"}, + {file = "coverage-7.6.9-cp313-cp313-win_amd64.whl", hash = "sha256:fd1213c86e48dfdc5a0cc676551db467495a95a662d2396ecd58e719191446e1"}, + {file = "coverage-7.6.9-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:ba9e7484d286cd5a43744e5f47b0b3fb457865baf07bafc6bee91896364e1419"}, + {file = "coverage-7.6.9-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e5ea1cf0872ee455c03e5674b5bca5e3e68e159379c1af0903e89f5eba9ccc3a"}, + {file = "coverage-7.6.9-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2d10e07aa2b91835d6abec555ec8b2733347956991901eea6ffac295f83a30e4"}, + {file = "coverage-7.6.9-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:13a9e2d3ee855db3dd6ea1ba5203316a1b1fd8eaeffc37c5b54987e61e4194ae"}, + {file = "coverage-7.6.9-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c38bf15a40ccf5619fa2fe8f26106c7e8e080d7760aeccb3722664c8656b030"}, + {file = "coverage-7.6.9-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:d5275455b3e4627c8e7154feaf7ee0743c2e7af82f6e3b561967b1cca755a0be"}, + {file = "coverage-7.6.9-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:8f8770dfc6e2c6a2d4569f411015c8d751c980d17a14b0530da2d7f27ffdd88e"}, + {file = "coverage-7.6.9-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:8d2dfa71665a29b153a9681edb1c8d9c1ea50dfc2375fb4dac99ea7e21a0bcd9"}, + {file = "coverage-7.6.9-cp313-cp313t-win32.whl", hash = "sha256:5e6b86b5847a016d0fbd31ffe1001b63355ed309651851295315031ea7eb5a9b"}, + {file = "coverage-7.6.9-cp313-cp313t-win_amd64.whl", hash = "sha256:97ddc94d46088304772d21b060041c97fc16bdda13c6c7f9d8fcd8d5ae0d8611"}, + {file = "coverage-7.6.9-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:adb697c0bd35100dc690de83154627fbab1f4f3c0386df266dded865fc50a902"}, + {file = "coverage-7.6.9-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:be57b6d56e49c2739cdf776839a92330e933dd5e5d929966fbbd380c77f060be"}, + {file = "coverage-7.6.9-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1592791f8204ae9166de22ba7e6705fa4ebd02936c09436a1bb85aabca3e599"}, + {file = "coverage-7.6.9-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4e12ae8cc979cf83d258acb5e1f1cf2f3f83524d1564a49d20b8bec14b637f08"}, + {file = "coverage-7.6.9-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb5555cff66c4d3d6213a296b360f9e1a8e323e74e0426b6c10ed7f4d021e464"}, + {file = "coverage-7.6.9-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:b9389a429e0e5142e69d5bf4a435dd688c14478a19bb901735cdf75e57b13845"}, + {file = "coverage-7.6.9-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:592ac539812e9b46046620341498caf09ca21023c41c893e1eb9dbda00a70cbf"}, + {file = "coverage-7.6.9-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:a27801adef24cc30871da98a105f77995e13a25a505a0161911f6aafbd66e678"}, + {file = "coverage-7.6.9-cp39-cp39-win32.whl", hash = "sha256:8e3c3e38930cfb729cb8137d7f055e5a473ddaf1217966aa6238c88bd9fd50e6"}, + {file = "coverage-7.6.9-cp39-cp39-win_amd64.whl", hash = "sha256:e28bf44afa2b187cc9f41749138a64435bf340adfcacb5b2290c070ce99839d4"}, + {file = "coverage-7.6.9-pp39.pp310-none-any.whl", hash = "sha256:f3ca78518bc6bc92828cd11867b121891d75cae4ea9e908d72030609b996db1b"}, + {file = "coverage-7.6.9.tar.gz", hash = "sha256:4a8d8977b0c6ef5aeadcb644da9e69ae0dcfe66ec7f368c89c72e058bd71164d"}, ] [package.dependencies] @@ -4649,13 +4649,13 @@ files = [ [[package]] name = "types-python-dateutil" -version = "2.9.0.20241003" +version = "2.9.0.20241206" description = "Typing stubs for python-dateutil" optional = false python-versions = ">=3.8" files = [ - {file = "types-python-dateutil-2.9.0.20241003.tar.gz", hash = "sha256:58cb85449b2a56d6684e41aeefb4c4280631246a0da1a719bdbe6f3fb0317446"}, - {file = "types_python_dateutil-2.9.0.20241003-py3-none-any.whl", hash = "sha256:250e1d8e80e7bbc3a6c99b907762711d1a1cdd00e978ad39cb5940f6f0a87f3d"}, + {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"}, ] [[package]] From a01c0b50495f9bf1366a3e25b892bb4fdfeb1c30 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Mon, 9 Dec 2024 15:08:10 +0200 Subject: [PATCH 09/34] Fix mypy error --- .../declarative/incremental/concurrent_partition_cursor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index dc89e5ddb..4d1263dd7 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -225,7 +225,7 @@ def _to_partition_key(self, partition: Mapping[str, Any]) -> str: def _to_dict(self, partition_key: str) -> Mapping[str, Any]: return self._partition_serializer.to_partition(partition_key) - def _create_cursor(self, cursor_state: Any) -> DeclarativeCursor: + def _create_cursor(self, cursor_state: Any) -> Cursor: cursor = self._cursor_factory.create(stream_state=cursor_state) return cursor From 357a925aaf5d282fb6b137d1636acd316ac1b0dc Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 18 Dec 2024 18:15:19 +0200 Subject: [PATCH 10/34] Add global cursor with fallback --- .../concurrent_partition_cursor.py | 53 +- .../parsers/model_to_component_factory.py | 5 +- .../sources/streams/concurrent/cursor.py | 10 +- .../test_concurrent_perpartitioncursor.py | 1032 +++++++++++++++++ 4 files changed, 1086 insertions(+), 14 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 4d1263dd7..2e1da77f6 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -3,11 +3,13 @@ # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # +import threading import logging from collections import OrderedDict from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import iterate_with_last_flag_and_state, Timer from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter from airbyte_cdk.sources.message import MessageRepository @@ -77,6 +79,15 @@ def __init__( # The dict is ordered to ensure that once the maximum number of partitions is reached, # the oldest partitions can be efficiently removed, maintaining the most recent partitions. self._cursor_per_partition: OrderedDict[str, Cursor] = OrderedDict() + self._state = {"states": []} + self._semaphore_per_partition = OrderedDict() + self._finished_partitions = set() + self._lock = threading.Lock() + self._timer = Timer() + self._global_cursor = None + self._new_global_cursor = None + self._lookback_window = 0 + self._parent_state = None self._over_limit = 0 self._partition_serializer = PerPartitionKeySerializer() @@ -91,7 +102,7 @@ def state(self) -> MutableMapping[str, Any]: states = [] for partition_tuple, cursor in self._cursor_per_partition.items(): cursor_state = cursor._connector_state_converter.convert_to_state_message( - cursor._cursor_field, cursor.state + self.cursor_field, cursor.state ) if cursor_state: states.append( @@ -101,16 +112,40 @@ def state(self) -> MutableMapping[str, Any]: } ) state: dict[str, Any] = {"states": states} + + state["state"] = self._global_cursor + if self._lookback_window is not None: + state["lookback_window"] = self._lookback_window + if self._parent_state is not None: + state["parent_state"] = self._parent_state + print(state) return state def close_partition(self, partition: Partition) -> None: - self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)].close_partition_without_emit(partition=partition) + print(f"Closing partition {self._to_partition_key(partition._stream_slice.partition)}") + self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)].close_partition(partition=partition) + with (self._lock): + self._semaphore_per_partition[self._to_partition_key(partition._stream_slice.partition)].acquire() + cursor = self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)] + cursor_state = cursor._connector_state_converter.convert_to_state_message( + cursor._cursor_field, cursor.state + ) + print(f"State {cursor_state} {cursor.state}") + if self._to_partition_key(partition._stream_slice.partition) in self._finished_partitions \ + and self._semaphore_per_partition[self._to_partition_key(partition._stream_slice.partition)]._value == 0: + if self._new_global_cursor is None \ + or self._new_global_cursor[self.cursor_field.cursor_field_key] < cursor_state[self.cursor_field.cursor_field_key]: + self._new_global_cursor = copy.deepcopy(cursor_state) def ensure_at_least_one_state_emitted(self) -> None: """ The platform expect to have at least one state message on successful syncs. Hence, whatever happens, we expect this method to be called. """ + if not any(semaphore_item[1]._value for semaphore_item in self._semaphore_per_partition.items()): + self._global_cursor = self._new_global_cursor + self._lookback_window = self._timer.finish() + self._parent_state = self._partition_router.get_stream_state() self._emit_state_message() def _emit_state_message(self) -> None: @@ -127,6 +162,7 @@ def _emit_state_message(self) -> None: def stream_slices(self) -> Iterable[StreamSlice]: slices = self._partition_router.stream_slices() + self._timer.start() for partition in slices: yield from self.generate_slices_from_partition(partition) @@ -143,8 +179,15 @@ def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[Str ) cursor = self._create_cursor(partition_state) self._cursor_per_partition[self._to_partition_key(partition.partition)] = cursor - - for cursor_slice in cursor.stream_slices(): + self._semaphore_per_partition[self._to_partition_key(partition.partition)] = threading.Semaphore(0) + + for cursor_slice, is_last_slice, _ in iterate_with_last_flag_and_state( + cursor.stream_slices(), + lambda: None, + ): + self._semaphore_per_partition[self._to_partition_key(partition.partition)].release() + if is_last_slice: + self._finished_partitions.add(self._to_partition_key(partition.partition)) yield StreamSlice( partition=partition, cursor_slice=cursor_slice, extra_fields=partition.extra_fields ) @@ -208,6 +251,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( self._create_cursor(state["cursor"]) ) + self._semaphore_per_partition[self._to_partition_key(state["partition"])] = threading.Semaphore(0) # set default state for missing partitions if it is per partition with fallback to global if "state" in stream_state: @@ -217,6 +261,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._partition_router.set_initial_state(stream_state) def observe(self, record: Record) -> None: + print(self._to_partition_key(record.associated_slice.partition), record) self._cursor_per_partition[self._to_partition_key(record.associated_slice.partition)].observe(record) def _to_partition_key(self, partition: Mapping[str, Any]) -> str: 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 e2663c92a..82bd80b90 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -376,6 +376,7 @@ InMemoryMessageRepository, LogAppenderMessageRepositoryDecorator, MessageRepository, + NoopMessageRepository, ) from airbyte_cdk.sources.streams.concurrent.cursor import ConcurrentCursor, CursorField from airbyte_cdk.sources.streams.concurrent.state_converters.datetime_stream_state_converter import ( @@ -773,6 +774,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( stream_namespace: Optional[str], config: Config, stream_state: MutableMapping[str, Any], + message_repository: Optional[MessageRepository] = None, **kwargs: Any, ) -> ConcurrentCursor: component_type = component_definition.get("type") @@ -908,7 +910,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( stream_name=stream_name, stream_namespace=stream_namespace, stream_state=stream_state, - message_repository=self._message_repository, + message_repository=message_repository or self._message_repository, connector_state_manager=state_manager, connector_state_converter=connector_state_converter, cursor_field=cursor_field, @@ -961,6 +963,7 @@ def create_concurrent_cursor_from_perpartition_cursor( stream_name=stream_name, stream_namespace=stream_namespace, config=config, + message_repository=NoopMessageRepository() ) ) diff --git a/airbyte_cdk/sources/streams/concurrent/cursor.py b/airbyte_cdk/sources/streams/concurrent/cursor.py index 43b2ffe1a..ea6b01928 100644 --- a/airbyte_cdk/sources/streams/concurrent/cursor.py +++ b/airbyte_cdk/sources/streams/concurrent/cursor.py @@ -226,6 +226,7 @@ def _get_concurrent_state( ) def observe(self, record: Record) -> None: + print(f"Observing record: {record}") most_recent_cursor_value = self._most_recent_cursor_value_per_partition.get( record.associated_slice ) @@ -240,15 +241,6 @@ def observe(self, record: Record) -> None: def _extract_cursor_value(self, record: Record) -> Any: return self._connector_state_converter.parse_value(self._cursor_field.extract_value(record)) - def close_partition_without_emit(self, partition: Partition) -> None: - slice_count_before = len(self.state.get("slices", [])) - self._add_slice_to_state(partition) - if slice_count_before < len( - self.state["slices"] - ): # only emit if at least one slice has been processed - self._merge_partitions() - self._has_closed_at_least_one_slice = True - def close_partition(self, partition: Partition) -> None: slice_count_before = len(self.state.get("slices", [])) self._add_slice_to_state(partition) diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index bbbcfdc2a..ab713a404 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -2,6 +2,7 @@ from typing import Any, List, Mapping, MutableMapping, Optional, Union from unittest.mock import MagicMock +import copy import pytest import requests_mock @@ -473,6 +474,11 @@ def _run_read( "cursor": {"created_at": "2024-01-10T00:00:00Z"}, }, ], + 'lookback_window': 1, + 'parent_state': {}, + 'state': {'created_at': '2024-01-15T00:00:00Z'} + + }, ), ], @@ -521,3 +527,1029 @@ def test_incremental_parent_state_no_incremental_dependency( if message.state ] assert final_state[-1] == expected_state + + +def run_incremental_parent_state_test( + manifest, mock_requests, expected_records, initial_state, expected_states +): + """ + Run an incremental parent state test for the specified stream. + + This function performs the following steps: + 1. Mocks the API requests as defined in mock_requests. + 2. Executes the read operation using the provided manifest and config. + 3. Asserts that the output records match the expected records. + 4. Collects intermediate states and records, performing additional reads as necessary. + 5. Compares the cumulative records from each state against the expected records. + 6. Asserts that the final state matches one of the expected states for each run. + + Args: + manifest (dict): The manifest configuration for the stream. + mock_requests (list): A list of tuples containing URL and response data for mocking API requests. + expected_records (list): The expected records to compare against the output. + initial_state (list): The initial state to start the read operation. + expected_states (list): A list of expected final states after the read operation. + """ + _stream_name = "post_comment_votes" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + # Run the initial read + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + # Assert that output_data equals expected_records + assert (sorted(output_data, key=lambda x: orjson.dumps(x)) + == sorted(expected_records, key=lambda x: orjson.dumps(x))) + + # Collect the intermediate states and records produced before each state + cumulative_records = [] + intermediate_states = [] + final_states = [] # To store the final state after each read + + # Store the final state after the initial read + final_state_initial = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + final_states.append(final_state_initial[-1]) + + for message in output: + if message.type.value == "RECORD": + record_data = message.record.data + cumulative_records.append(record_data) + elif message.type.value == "STATE": + # Record the state and the records produced before this state + state = message.state + records_before_state = cumulative_records.copy() + intermediate_states.append((state, records_before_state)) + + # For each intermediate state, perform another read starting from that state + for state, records_before_state in intermediate_states[:-1]: + output_intermediate = _run_read(manifest, config, _stream_name, [state]) + records_from_state = [ + message.record.data for message in output_intermediate if message.record + ] + + # Combine records produced before the state with records from the new read + cumulative_records_state = records_before_state + records_from_state + + # Duplicates may occur because the state matches the cursor of the last record, causing it to be re-emitted in the next sync. + cumulative_records_state_deduped = list( + {orjson.dumps(record): record for record in cumulative_records_state}.values() + ) + + # Compare the cumulative records with the expected records + expected_records_set = list( + {orjson.dumps(record): record for record in expected_records}.values() + ) + assert ( + sorted(cumulative_records_state_deduped, key=lambda x: orjson.dumps(x)) + == sorted(expected_records_set, key=lambda x: orjson.dumps(x)) + ), f"Records mismatch with intermediate state {state}. Expected {expected_records}, got {cumulative_records_state_deduped}" + + # Store the final state after each intermediate read + final_state_intermediate = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output_intermediate + if message.state + ] + final_states.append(final_state_intermediate[-1]) + + # Assert that the final state matches the expected state for all runs + for i, final_state in enumerate(final_states): + assert ( + final_state in expected_states + ), f"Final state mismatch at run {i + 1}. Expected {expected_states}, got {final_state}" + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + SUBSTREAM_MANIFEST, + [ + # Fetch the first page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + { + "posts": [ + {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, + {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + }, + ), + # Fetch the second page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + }, + ), + # Fetch the second page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + { + "votes": [ + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} + ] + }, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 30 of post 3 + ( + "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + { + "votes": [ + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} + ] + }, + ), + # Requests with intermediate states + # Fetch votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-15T00:00:00Z", + { + "votes": [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + ], + }, + ), + # Fetch votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-13T00:00:00Z", + { + "votes": [ + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} + ], + }, + ), + # Fetch votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-15T00:00:00Z", + { + "votes": [], + }, + ), + # Fetch votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-12T00:00:00Z", + { + "votes": [ + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} + ] + }, + ), + # Fetch votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-12T00:00:15Z", + { + "votes": [ + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} + ] + }, + ), + ], + # Expected records + [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"}, + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + ], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + } + ], + "parent_state": { + "posts": {"updated_at": "2024-01-05T00:00:00Z"} + }, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + ], + } + ), + ), + ) + ], + # Expected state + { + "state": {"created_at": "2024-01-15T00:00:00Z"}, + "parent_state": { + "post_comments": { + "use_global_cursor": False, + "state": {"updated_at": "2024-01-25T00:00:00Z"}, + "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + }, + { + "partition": {"id": 2, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + }, + { + "partition": {"id": 3, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + }, + ], + } + }, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + }, + { + "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + }, + { + 'partition': {'id': 12, 'parent_slice': {'id': 1, 'parent_slice': {}}}, + 'cursor': {'created_at': '2024-01-01T00:00:01Z'}, + }, + { + "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:00Z"}, + }, + { + "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + }, + { + "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + }, + ], + }, + ), + ], +) +def test_incremental_parent_state( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + additional_expected_state = copy.deepcopy(expected_state) + # State for empty partition (comment 12), when the global cursor is used for intermediate states + empty_state = { + "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + } + additional_expected_state["states"].append(empty_state) + run_incremental_parent_state_test( + manifest, + mock_requests, + expected_records, + initial_state, + [expected_state, additional_expected_state], + ) + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + SUBSTREAM_MANIFEST, + [ + # Fetch the first page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "posts": [ + {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, + {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-02T00:00:00Z&page=2", + }, + ), + # Fetch the second page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-02T00:00:00Z&page=2", + {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-02T00:00:00Z", + }, + ), + # Fetch the second page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} + ] + }, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 30 of post 3 + ( + "https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} + ] + }, + ), + ], + # Expected records + [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, + {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"}, + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + ], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob({"created_at": "2024-01-02T00:00:00Z"}), + ), + ) + ], + # Expected state + { + "use_global_cursor": False, + "state": {"created_at": "2024-01-15T00:00:00Z"}, + "parent_state": { + "post_comments": { + "use_global_cursor": False, + "state": {"updated_at": "2024-01-25T00:00:00Z"}, + "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + }, + { + "partition": {"id": 2, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + }, + { + "partition": {"id": 3, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + }, + ], + } + }, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + }, + { + "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + }, + { + "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:00Z"}, + }, + { + "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + }, + { + "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + }, + ], + }, + ), + ], +) +def test_incremental_parent_state_migration( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + """ + Test incremental partition router with parent state migration + """ + _stream_name = "post_comment_votes" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert set(tuple(sorted(d.items())) for d in output_data) == set(tuple(sorted(d.items())) for d in expected_records) + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + SUBSTREAM_MANIFEST, + [ + # Fetch the first page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + { + "posts": [], + "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + }, + ), + # Fetch the second page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + {"posts": []}, + ), + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": []}, + ), + # Fetch the first page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [], + "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + }, + ), + # Fetch the second page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + {"votes": []}, + ), + # Fetch the first page of votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": []}, + ), + # Fetch the first page of votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": []}, + ), + # Fetch the first page of votes for comment 30 of post 3 + ( + "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + {"votes": []}, + ), + ], + # Expected records + [], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + } + ], + "parent_state": { + "posts": {"updated_at": "2024-01-05T00:00:00Z"} + }, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + ], + "state": {"created_at": "2024-01-03T00:00:00Z"}, + "lookback_window": 1, + } + ), + ), + ) + ], + # Expected state + { + "lookback_window": 1, + "use_global_cursor": False, + "state": {"created_at": "2024-01-03T00:00:00Z"}, + "parent_state": { + "post_comments": { + "use_global_cursor": False, + "state": {}, + "parent_state": {"posts": {"updated_at": "2024-01-05T00:00:00Z"}}, + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + } + ], + } + }, + "states": [ + { + "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + ], + }, + ), + ], +) +def test_incremental_parent_state_no_slices( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + """ + Test incremental partition router with no parent records + """ + _stream_name = "post_comment_votes" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert output_data == expected_records + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + SUBSTREAM_MANIFEST, + [ + # Fetch the first page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + { + "posts": [ + {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, + {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + }, + ), + # Fetch the second page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + { + "votes": [], + "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + }, + ), + # Fetch the second page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + {"votes": []}, + ), + # Fetch the first page of votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + {"votes": []}, + ), + # Fetch the first page of votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 30 of post 3 + ( + "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + {"votes": []}, + ), + ], + # Expected records + [], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + } + ], + "parent_state": { + "posts": {"updated_at": "2024-01-05T00:00:00Z"} + }, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + ], + "use_global_cursor": True, + "state": {"created_at": "2024-01-03T00:00:00Z"}, + "lookback_window": 0, + } + ), + ), + ) + ], + # Expected state + { + "lookback_window": 1, + "use_global_cursor": True, + "state": {"created_at": "2024-01-03T00:00:00Z"}, + "parent_state": { + "post_comments": { + "use_global_cursor": False, + "state": {"updated_at": "2024-01-25T00:00:00Z"}, + "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + }, + { + "partition": {"id": 2, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + }, + { + "partition": {"id": 3, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + }, + ], + } + }, + }, + ), + ], +) +def test_incremental_parent_state_no_records( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + """ + Test incremental partition router with no child records + """ + _stream_name = "post_comment_votes" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert output_data == expected_records + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state From a36726bb5eb0cebc4fa2cadc6c57df5a99603a47 Mon Sep 17 00:00:00 2001 From: octavia-squidington-iii Date: Wed, 18 Dec 2024 16:19:36 +0000 Subject: [PATCH 11/34] Auto-fix lint and format issues --- .../concurrent_declarative_source.py | 23 ++++---- .../concurrent_partition_cursor.py | 59 +++++++++++++------ .../incremental/per_partition_cursor.py | 9 ++- .../parsers/model_to_component_factory.py | 36 +++++------ .../declarative_partition_generator.py | 1 + .../test_concurrent_perpartitioncursor.py | 31 +++++----- 6 files changed, 96 insertions(+), 63 deletions(-) diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index dc99c4149..1b607870e 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -311,7 +311,9 @@ def _group_streams( declarative_stream=declarative_stream ) and hasattr(declarative_stream.retriever, "stream_slicer") - and isinstance(declarative_stream.retriever.stream_slicer, PerPartitionWithGlobalCursor) + and isinstance( + declarative_stream.retriever.stream_slicer, PerPartitionWithGlobalCursor + ) ): stream_state = state_manager.get_stream_state( stream_name=declarative_stream.name, namespace=declarative_stream.namespace @@ -319,16 +321,15 @@ def _group_streams( partition_router = declarative_stream.retriever.stream_slicer._partition_router cursor = self._constructor.create_concurrent_cursor_from_perpartition_cursor( - state_manager=state_manager, - model_type=DatetimeBasedCursorModel, - component_definition=incremental_sync_component_definition, - stream_name=declarative_stream.name, - stream_namespace=declarative_stream.namespace, - config=config or {}, - stream_state=stream_state, - partition_router=partition_router, - ) - + state_manager=state_manager, + model_type=DatetimeBasedCursorModel, + component_definition=incremental_sync_component_definition, + stream_name=declarative_stream.name, + stream_namespace=declarative_stream.namespace, + config=config or {}, + stream_state=stream_state, + partition_router=partition_router, + ) partition_generator = StreamSlicerPartitionGenerator( DeclarativePartitionFactory( diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 2e1da77f6..e43134f0a 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -1,16 +1,19 @@ import copy +import logging # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # import threading -import logging from collections import OrderedDict from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager -from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import iterate_with_last_flag_and_state, Timer from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor +from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( + Timer, + iterate_with_last_flag_and_state, +) from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter from airbyte_cdk.sources.message import MessageRepository from airbyte_cdk.sources.streams.checkpoint.per_partition_key_serializer import ( @@ -123,18 +126,33 @@ def state(self) -> MutableMapping[str, Any]: def close_partition(self, partition: Partition) -> None: print(f"Closing partition {self._to_partition_key(partition._stream_slice.partition)}") - self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)].close_partition(partition=partition) - with (self._lock): - self._semaphore_per_partition[self._to_partition_key(partition._stream_slice.partition)].acquire() - cursor = self._cursor_per_partition[self._to_partition_key(partition._stream_slice.partition)] + self._cursor_per_partition[ + self._to_partition_key(partition._stream_slice.partition) + ].close_partition(partition=partition) + with self._lock: + self._semaphore_per_partition[ + self._to_partition_key(partition._stream_slice.partition) + ].acquire() + cursor = self._cursor_per_partition[ + self._to_partition_key(partition._stream_slice.partition) + ] cursor_state = cursor._connector_state_converter.convert_to_state_message( cursor._cursor_field, cursor.state ) print(f"State {cursor_state} {cursor.state}") - if self._to_partition_key(partition._stream_slice.partition) in self._finished_partitions \ - and self._semaphore_per_partition[self._to_partition_key(partition._stream_slice.partition)]._value == 0: - if self._new_global_cursor is None \ - or self._new_global_cursor[self.cursor_field.cursor_field_key] < cursor_state[self.cursor_field.cursor_field_key]: + if ( + self._to_partition_key(partition._stream_slice.partition) + in self._finished_partitions + and self._semaphore_per_partition[ + self._to_partition_key(partition._stream_slice.partition) + ]._value + == 0 + ): + if ( + self._new_global_cursor is None + or self._new_global_cursor[self.cursor_field.cursor_field_key] + < cursor_state[self.cursor_field.cursor_field_key] + ): self._new_global_cursor = copy.deepcopy(cursor_state) def ensure_at_least_one_state_emitted(self) -> None: @@ -142,7 +160,9 @@ def ensure_at_least_one_state_emitted(self) -> None: The platform expect to have at least one state message on successful syncs. Hence, whatever happens, we expect this method to be called. """ - if not any(semaphore_item[1]._value for semaphore_item in self._semaphore_per_partition.items()): + if not any( + semaphore_item[1]._value for semaphore_item in self._semaphore_per_partition.items() + ): self._global_cursor = self._new_global_cursor self._lookback_window = self._timer.finish() self._parent_state = self._partition_router.get_stream_state() @@ -159,7 +179,6 @@ def _emit_state_message(self) -> None: ) self._message_repository.emit_message(state_message) - def stream_slices(self) -> Iterable[StreamSlice]: slices = self._partition_router.stream_slices() self._timer.start() @@ -179,11 +198,13 @@ def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[Str ) cursor = self._create_cursor(partition_state) self._cursor_per_partition[self._to_partition_key(partition.partition)] = cursor - self._semaphore_per_partition[self._to_partition_key(partition.partition)] = threading.Semaphore(0) + self._semaphore_per_partition[self._to_partition_key(partition.partition)] = ( + threading.Semaphore(0) + ) for cursor_slice, is_last_slice, _ in iterate_with_last_flag_and_state( - cursor.stream_slices(), - lambda: None, + cursor.stream_slices(), + lambda: None, ): self._semaphore_per_partition[self._to_partition_key(partition.partition)].release() if is_last_slice: @@ -251,7 +272,9 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( self._create_cursor(state["cursor"]) ) - self._semaphore_per_partition[self._to_partition_key(state["partition"])] = threading.Semaphore(0) + self._semaphore_per_partition[self._to_partition_key(state["partition"])] = ( + threading.Semaphore(0) + ) # set default state for missing partitions if it is per partition with fallback to global if "state" in stream_state: @@ -262,7 +285,9 @@ def _set_initial_state(self, stream_state: StreamState) -> None: def observe(self, record: Record) -> None: print(self._to_partition_key(record.associated_slice.partition), record) - self._cursor_per_partition[self._to_partition_key(record.associated_slice.partition)].observe(record) + self._cursor_per_partition[ + self._to_partition_key(record.associated_slice.partition) + ].observe(record) def _to_partition_key(self, partition: Mapping[str, Any]) -> str: return self._partition_serializer.to_partition_key(partition) diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py index d7322709d..1529e90ec 100644 --- a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py @@ -303,7 +303,10 @@ def get_request_body_json( raise ValueError("A partition needs to be provided in order to get request body json") def should_be_synced(self, record: Record) -> bool: - if self._to_partition_key(record.associated_slice.partition) not in self._cursor_per_partition: + if ( + self._to_partition_key(record.associated_slice.partition) + not in self._cursor_per_partition + ): partition_state = ( self._state_to_migrate_from if self._state_to_migrate_from @@ -311,7 +314,9 @@ def should_be_synced(self, record: Record) -> bool: ) cursor = self._create_cursor(partition_state) - self._cursor_per_partition[self._to_partition_key(record.associated_slice.partition)] = cursor + self._cursor_per_partition[ + self._to_partition_key(record.associated_slice.partition) + ] = cursor return self._get_cursor(record).should_be_synced( self._convert_record_to_cursor_record(record) ) 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 d210b475c..2decfbd4f 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -969,7 +969,7 @@ def create_concurrent_cursor_from_perpartition_cursor( config: Config, stream_state: MutableMapping[str, Any], partition_router, - **kwargs: Any, + **kwargs: Any, ) -> ConcurrentPerPartitionCursor: component_type = component_definition.get("type") if component_definition.get("type") != model_type.__name__: @@ -1000,21 +1000,21 @@ def create_concurrent_cursor_from_perpartition_cursor( stream_name=stream_name, stream_namespace=stream_namespace, config=config, - message_repository=NoopMessageRepository() + message_repository=NoopMessageRepository(), ) ) # Return the concurrent cursor and state converter return ConcurrentPerPartitionCursor( - cursor_factory=cursor_factory, - partition_router=partition_router, - stream_name=stream_name, - stream_namespace=stream_namespace, - stream_state=stream_state, - message_repository=self._message_repository, # type: ignore - connector_state_manager=state_manager, - cursor_field=cursor_field, - ) + cursor_factory=cursor_factory, + partition_router=partition_router, + stream_name=stream_name, + stream_namespace=stream_namespace, + stream_state=stream_state, + message_repository=self._message_repository, # type: ignore + connector_state_manager=state_manager, + cursor_field=cursor_field, + ) @staticmethod def create_constant_backoff_strategy( @@ -1298,15 +1298,15 @@ def create_declarative_stream( raise ValueError( "Unsupported Slicer is used. PerPartitionWithGlobalCursor should be used here instead" ) - cursor = combined_slicers if isinstance( - combined_slicers, (PerPartitionWithGlobalCursor, GlobalSubstreamCursor) - ) else self._create_component_from_model( - model=model.incremental_sync, config=config + cursor = ( + combined_slicers + if isinstance( + combined_slicers, (PerPartitionWithGlobalCursor, GlobalSubstreamCursor) + ) + else self._create_component_from_model(model=model.incremental_sync, config=config) ) - client_side_incremental_sync = { - "cursor": cursor - } + client_side_incremental_sync = {"cursor": cursor} if model.incremental_sync and isinstance(model.incremental_sync, DatetimeBasedCursorModel): cursor_model = model.incremental_sync 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 09ed2bc80..31f6377f7 100644 --- a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py +++ b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py @@ -38,6 +38,7 @@ def create(self, stream_slice: StreamSlice) -> Partition: stream_slice, ) + class DeclarativePartition(Partition): def __init__( self, diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index ab713a404..57948d79c 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -1,8 +1,8 @@ # Copyright (c) 2024 Airbyte, Inc., all rights reserved. +import copy from typing import Any, List, Mapping, MutableMapping, Optional, Union from unittest.mock import MagicMock -import copy import pytest import requests_mock @@ -264,9 +264,7 @@ def _run_read( source = ConcurrentDeclarativeSource( source_config=manifest, config=config, catalog=catalog, state=state ) - messages = list( - source.read(logger=source.logger, config=config, catalog=catalog, state=[]) - ) + messages = list(source.read(logger=source.logger, config=config, catalog=catalog, state=[])) return messages @@ -474,11 +472,9 @@ def _run_read( "cursor": {"created_at": "2024-01-10T00:00:00Z"}, }, ], - 'lookback_window': 1, - 'parent_state': {}, - 'state': {'created_at': '2024-01-15T00:00:00Z'} - - + "lookback_window": 1, + "parent_state": {}, + "state": {"created_at": "2024-01-15T00:00:00Z"}, }, ), ], @@ -520,7 +516,9 @@ def test_incremental_parent_state_no_incremental_dependency( output = _run_read(manifest, config, _stream_name, initial_state) output_data = [message.record.data for message in output if message.record] - assert set(tuple(sorted(d.items())) for d in output_data) == set(tuple(sorted(d.items())) for d in expected_records) + assert set(tuple(sorted(d.items())) for d in output_data) == set( + tuple(sorted(d.items())) for d in expected_records + ) final_state = [ orjson.loads(orjson.dumps(message.state.stream.stream_state)) for message in output @@ -565,8 +563,9 @@ def run_incremental_parent_state_test( output_data = [message.record.data for message in output if message.record] # Assert that output_data equals expected_records - assert (sorted(output_data, key=lambda x: orjson.dumps(x)) - == sorted(expected_records, key=lambda x: orjson.dumps(x))) + assert sorted(output_data, key=lambda x: orjson.dumps(x)) == sorted( + expected_records, key=lambda x: orjson.dumps(x) + ) # Collect the intermediate states and records produced before each state cumulative_records = [] @@ -884,8 +883,8 @@ def run_incremental_parent_state_test( "cursor": {"created_at": "2024-01-13T00:00:00Z"}, }, { - 'partition': {'id': 12, 'parent_slice': {'id': 1, 'parent_slice': {}}}, - 'cursor': {'created_at': '2024-01-01T00:00:01Z'}, + "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-01T00:00:01Z"}, }, { "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, @@ -1141,7 +1140,9 @@ def test_incremental_parent_state_migration( output = _run_read(manifest, config, _stream_name, initial_state) output_data = [message.record.data for message in output if message.record] - assert set(tuple(sorted(d.items())) for d in output_data) == set(tuple(sorted(d.items())) for d in expected_records) + assert set(tuple(sorted(d.items())) for d in output_data) == set( + tuple(sorted(d.items())) for d in expected_records + ) final_state = [ orjson.loads(orjson.dumps(message.state.stream.stream_state)) for message in output From 5ee05f11d67e0cccdb819386449b22335a7dc8d0 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 18 Dec 2024 19:18:00 +0200 Subject: [PATCH 12/34] Fix parent state update in case of error --- .../declarative/incremental/concurrent_partition_cursor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index e43134f0a..d1c594aad 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -165,7 +165,7 @@ def ensure_at_least_one_state_emitted(self) -> None: ): self._global_cursor = self._new_global_cursor self._lookback_window = self._timer.finish() - self._parent_state = self._partition_router.get_stream_state() + self._parent_state = self._partition_router.get_stream_state() self._emit_state_message() def _emit_state_message(self) -> None: From 24268e212e46fcb13831ae2b7bc236a2d33af026 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 26 Dec 2024 17:45:54 +0200 Subject: [PATCH 13/34] Fix error in tests --- .../concurrent_partition_cursor.py | 24 +- .../test_concurrent_perpartitioncursor.py | 310 +++++++++++++++++- 2 files changed, 311 insertions(+), 23 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index d1c594aad..83b774fe5 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -6,6 +6,7 @@ # import threading from collections import OrderedDict +from copy import deepcopy from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager @@ -57,7 +58,6 @@ class ConcurrentPerPartitionCursor(Cursor): _NO_CURSOR_STATE: Mapping[str, Any] = {} _KEY = 0 _VALUE = 1 - _state_to_migrate_from: Mapping[str, Any] = {} def __init__( self, @@ -70,6 +70,7 @@ def __init__( connector_state_manager: ConnectorStateManager, cursor_field: CursorField, ) -> None: + self._global_cursor: Mapping[str, Any] = {} self._stream_name = stream_name self._stream_namespace = stream_namespace self._message_repository = message_repository @@ -87,7 +88,6 @@ def __init__( self._finished_partitions = set() self._lock = threading.Lock() self._timer = Timer() - self._global_cursor = None self._new_global_cursor = None self._lookback_window = 0 self._parent_state = None @@ -116,12 +116,12 @@ def state(self) -> MutableMapping[str, Any]: ) state: dict[str, Any] = {"states": states} - state["state"] = self._global_cursor + if self._global_cursor: + state["state"] = self._global_cursor if self._lookback_window is not None: state["lookback_window"] = self._lookback_window if self._parent_state is not None: state["parent_state"] = self._parent_state - print(state) return state def close_partition(self, partition: Partition) -> None: @@ -192,8 +192,8 @@ def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[Str cursor = self._cursor_per_partition.get(self._to_partition_key(partition.partition)) if not cursor: partition_state = ( - self._state_to_migrate_from - if self._state_to_migrate_from + self._global_cursor + if self._global_cursor else self._NO_CURSOR_STATE ) cursor = self._create_cursor(partition_state) @@ -265,7 +265,8 @@ def _set_initial_state(self, stream_state: StreamState) -> None: if "states" not in stream_state: # We assume that `stream_state` is in a global format that can be applied to all partitions. # Example: {"global_state_format_key": "global_state_format_value"} - self._state_to_migrate_from = stream_state + self._global_cursor = deepcopy(stream_state) + self._new_global_cursor = deepcopy(stream_state) else: for state in stream_state["states"]: @@ -278,13 +279,16 @@ def _set_initial_state(self, stream_state: StreamState) -> None: # set default state for missing partitions if it is per partition with fallback to global if "state" in stream_state: - self._state_to_migrate_from = stream_state["state"] + self._global_cursor = deepcopy(stream_state["state"]) + self._new_global_cursor = deepcopy(stream_state["state"]) # Set parent state for partition routers based on parent streams self._partition_router.set_initial_state(stream_state) def observe(self, record: Record) -> None: - print(self._to_partition_key(record.associated_slice.partition), record) + print("Observing record in concirrent perpartition ", self._to_partition_key(record.associated_slice.partition), record, self._cursor_per_partition[ + self._to_partition_key(record.associated_slice.partition) + ].state) self._cursor_per_partition[ self._to_partition_key(record.associated_slice.partition) ].observe(record) @@ -296,7 +300,7 @@ def _to_dict(self, partition_key: str) -> Mapping[str, Any]: return self._partition_serializer.to_partition(partition_key) def _create_cursor(self, cursor_state: Any) -> Cursor: - cursor = self._cursor_factory.create(stream_state=cursor_state) + cursor = self._cursor_factory.create(stream_state=deepcopy(cursor_state)) return cursor def should_be_synced(self, record: Record) -> bool: diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index 57948d79c..cdeb70d42 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -1,6 +1,7 @@ # Copyright (c) 2024 Airbyte, Inc., all rights reserved. import copy +from copy import deepcopy from typing import Any, List, Mapping, MutableMapping, Optional, Union from unittest.mock import MagicMock @@ -240,6 +241,16 @@ }, } +SUBSTREAM_MANIFEST_NO_DEPENDENCY = deepcopy(SUBSTREAM_MANIFEST) +# Disable incremental_dependency +SUBSTREAM_MANIFEST_NO_DEPENDENCY["definitions"]["post_comments_stream"]["retriever"]["partition_router"][ + "parent_stream_configs" +][0]["incremental_dependency"] = False +SUBSTREAM_MANIFEST_NO_DEPENDENCY["definitions"]["post_comment_votes_stream"]["retriever"]["partition_router"][ + "parent_stream_configs" +][0]["incremental_dependency"] = False + + def _run_read( manifest: Mapping[str, Any], @@ -273,7 +284,7 @@ def _run_read( [ ( "test_incremental_parent_state", - SUBSTREAM_MANIFEST, + SUBSTREAM_MANIFEST_NO_DEPENDENCY, [ # Fetch the first page of posts ( @@ -501,14 +512,6 @@ def test_incremental_parent_state_no_incremental_dependency( "credentials": {"email": "email", "api_token": "api_token"}, } - # Disable incremental_dependency - manifest["definitions"]["post_comments_stream"]["retriever"]["partition_router"][ - "parent_stream_configs" - ][0]["incremental_dependency"] = False - manifest["definitions"]["post_comment_votes_stream"]["retriever"]["partition_router"][ - "parent_stream_configs" - ][0]["incremental_dependency"] = False - with requests_mock.Mocker() as m: for url, response in mock_requests: m.get(url, json=response) @@ -913,6 +916,7 @@ def test_incremental_parent_state( "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, } additional_expected_state["states"].append(empty_state) + print(manifest) run_incremental_parent_state_test( manifest, mock_requests, @@ -1066,7 +1070,6 @@ def test_incremental_parent_state( ], # Expected state { - "use_global_cursor": False, "state": {"created_at": "2024-01-15T00:00:00Z"}, "parent_state": { "post_comments": { @@ -1292,7 +1295,6 @@ def test_incremental_parent_state_migration( # Expected state { "lookback_window": 1, - "use_global_cursor": False, "state": {"created_at": "2024-01-03T00:00:00Z"}, "parent_state": { "post_comments": { @@ -1391,7 +1393,7 @@ def test_incremental_parent_state_no_slices( ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", { "votes": [], "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", @@ -1500,8 +1502,35 @@ def test_incremental_parent_state_no_slices( # Expected state { "lookback_window": 1, - "use_global_cursor": True, "state": {"created_at": "2024-01-03T00:00:00Z"}, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, + 'partition': {'id': 12, + 'parent_slice': {'id': 1, 'parent_slice': {}}}}, + {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, + 'partition': {'id': 20, + 'parent_slice': {'id': 2, 'parent_slice': {}}}}, + {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, + 'partition': {'id': 21, + 'parent_slice': {'id': 2, 'parent_slice': {}}}}, + {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, + 'partition': {'id': 30, + 'parent_slice': {'id': 3, 'parent_slice': {}}}} + ], "parent_state": { "post_comments": { "use_global_cursor": False, @@ -1554,3 +1583,258 @@ def test_incremental_parent_state_no_records( if message.state ] assert final_state[-1] == expected_state + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + SUBSTREAM_MANIFEST, + [ + # Fetch the first page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + { + "posts": [ + {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, + {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + }, + ), + # Fetch the second page of posts + ( + "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + { + "votes": [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + }, + ), + # Fetch the second page of votes for comment 10 of post 1 + ( + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 11 of post 1 + ( + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + { + "votes": [ + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} + ] + }, + ), + # Fetch the first page of votes for comment 12 of post 1 + ( + "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + {"votes": []}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 20 of post 2 + ( + "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "status_code": 500, + "json": {"error": "Internal Server Error"}, + }, + ), + # Fetch the first page of votes for comment 21 of post 2 + ( + "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + { + "votes": [ + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} + ] + }, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + # Fetch the first page of votes for comment 30 of post 3 + ( + "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + { + "votes": [ + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} + ] + }, + ), + ], + # Expected records + [ + {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, + {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, + {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, + {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, + {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + ], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + } + ], + "parent_state": { + "posts": {"updated_at": "2024-01-05T00:00:00Z"} + }, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + }, + ], + } + ), + ), + ) + ], + # Expected state + { + "state": {"created_at": "2024-01-15T00:00:00Z"}, + "parent_state": { + "post_comments": { + "use_global_cursor": False, + "state": {"updated_at": "2024-01-25T00:00:00Z"}, + "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + }, + { + "partition": {"id": 2, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + }, + { + "partition": {"id": 3, "parent_slice": {}}, + "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + }, + ], + } + }, + "lookback_window": 1, + "states": [ + { + "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + }, + { + "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + }, + { + "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + }, + { + "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + }, + { + "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + }, + { + "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, + "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + }, + ], + }, + ), + ], +) +def test_incremental_error__parent_state( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + _stream_name = "post_comment_votes" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert set(tuple(sorted(d.items())) for d in output_data) == set( + tuple(sorted(d.items())) for d in expected_records + ) + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state From 660da93d1cdb9a57b19566d73be19d39ae9462ed Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Mon, 13 Jan 2025 18:43:11 +0200 Subject: [PATCH 14/34] Fix unit tests --- .../sources/streams/concurrent/cursor.py | 1 - .../test_concurrent_perpartitioncursor.py | 238 ++++++++++++++++++ .../test_model_to_component_factory.py | 2 +- .../test_concurrent_declarative_source.py | 29 +-- 4 files changed, 254 insertions(+), 16 deletions(-) diff --git a/airbyte_cdk/sources/streams/concurrent/cursor.py b/airbyte_cdk/sources/streams/concurrent/cursor.py index ea6b01928..cbce82a94 100644 --- a/airbyte_cdk/sources/streams/concurrent/cursor.py +++ b/airbyte_cdk/sources/streams/concurrent/cursor.py @@ -226,7 +226,6 @@ def _get_concurrent_state( ) def observe(self, record: Record) -> None: - print(f"Observing record: {record}") most_recent_cursor_value = self._most_recent_cursor_value_per_partition.get( record.associated_slice ) diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index cdeb70d42..989ff3158 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -1838,3 +1838,241 @@ def test_incremental_error__parent_state( if message.state ] assert final_state[-1] == expected_state + + +LISTPARTITION_MANIFEST: MutableMapping[str, Any] = { + "version": "0.51.42", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["post_comments"]}, + "definitions": { + "basic_authenticator": { + "type": "BasicHttpAuthenticator", + "username": "{{ config['credentials']['email'] + '/token' }}", + "password": "{{ config['credentials']['api_token'] }}", + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.example.com", + "http_method": "GET", + "authenticator": "#/definitions/basic_authenticator", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": { + "type": "DpathExtractor", + "field_path": ["{{ parameters.get('data_path') or parameters['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": "{{ not response.get('next_page', {}) }}", + }, + "page_token_option": {"type": "RequestPath"}, + }, + }, + "cursor_incremental_sync": { + "type": "DatetimeBasedCursor", + "cursor_datetime_formats": ["%Y-%m-%dT%H:%M:%SZ", "%Y-%m-%dT%H:%M:%S%z"], + "datetime_format": "%Y-%m-%dT%H:%M:%SZ", + "cursor_field": "{{ parameters.get('cursor_field', 'updated_at') }}", + "start_datetime": {"datetime": "{{ config.get('start_date')}}"}, + "start_time_option": { + "inject_into": "request_parameter", + "field_name": "start_time", + "type": "RequestOption", + }, + }, + "post_comments_stream": { + "type": "DeclarativeStream", + "name": "post_comments", + "primary_key": ["id"], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "id": {"type": "integer"}, + "updated_at": {"type": "string", "format": "date-time"}, + "post_id": {"type": "integer"}, + "comment": {"type": "string"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.example.com", + "path": "/community/posts/{{ stream_slice.id }}/comments", + "http_method": "GET", + "authenticator": "#/definitions/basic_authenticator", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": { + "type": "DpathExtractor", + "field_path": ["{{ parameters.get('data_path') or parameters['name'] }}"], + }, + "schema_normalization": "Default", + }, + "paginator": "#/definitions/retriever/paginator", + "partition_router": { + "type": "ListPartitionRouter", + "cursor_field": "id", + "values": ["1", "2", "3"], + }, + }, + "incremental_sync": { + "$ref": "#/definitions/cursor_incremental_sync", + "is_client_side_incremental": True + }, + "$parameters": { + "name": "post_comments", + "path": "community/posts/{{ stream_slice.id }}/comments", + "data_path": "comments", + "cursor_field": "updated_at", + "primary_key": "id", + }, + }, + }, + "streams": [ + {"$ref": "#/definitions/post_comments_stream"}, + ], + "concurrency_level": { + "type": "ConcurrencyLevel", + "default_concurrency": "{{ config['num_workers'] or 10 }}", + "max_concurrency": 25, + }, +} + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_parent_state", + LISTPARTITION_MANIFEST, + [ + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + ], + # Expected records + [ + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + {"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}, + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"}, + {"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}, + {"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}, + ], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name="post_comment_votes", namespace=None + ), + stream_state=AirbyteStateBlob( + { + 'state': {'updated_at': '2024-01-10T00:00:00Z'}, + 'states': [{'cursor': {'updated_at': '2024-01-25T00:00:00Z'}, + 'partition': {'id': '1'}}, + {'cursor': {'updated_at': '2024-01-22T00:00:00Z'}, + 'partition': {'id': '2'}}, + {'cursor': {'updated_at': '2024-01-09T00:00:00Z'}, + 'partition': {'id': '3'}}], + 'use_global_cursor': False} + ), + ), + ) + ], + # Expected state + {'lookback_window': 1, + 'state': {'updated_at': '2024-01-25T00:00:00Z'}, + 'states': [{'cursor': {'updated_at': '2024-01-25T00:00:00Z'}, + 'partition': {'id': '1'}}, + {'cursor': {'updated_at': '2024-01-22T00:00:00Z'}, + 'partition': {'id': '2'}}, + {'cursor': {'updated_at': '2024-01-09T00:00:00Z'}, + 'partition': {'id': '3'}}], + }, + ), + ], +) +def test_incremental_list_partition_router( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + """ + Test ConcurrentPerPartitionCursor with ListPartitionRouter + """ + + _stream_name = "post_comments" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert set(tuple(sorted(d.items())) for d in output_data) == set( + tuple(sorted(d.items())) for d in expected_records + ) + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state diff --git a/unit_tests/sources/declarative/parsers/test_model_to_component_factory.py b/unit_tests/sources/declarative/parsers/test_model_to_component_factory.py index e849af853..06b716d47 100644 --- a/unit_tests/sources/declarative/parsers/test_model_to_component_factory.py +++ b/unit_tests/sources/declarative/parsers/test_model_to_component_factory.py @@ -1249,7 +1249,7 @@ def test_client_side_incremental_with_partition_router(): stream.retriever.record_selector.record_filter, ClientSideIncrementalRecordFilterDecorator ) assert isinstance( - stream.retriever.record_selector.record_filter._substream_cursor, + stream.retriever.record_selector.record_filter._cursor, PerPartitionWithGlobalCursor, ) diff --git a/unit_tests/sources/declarative/test_concurrent_declarative_source.py b/unit_tests/sources/declarative/test_concurrent_declarative_source.py index 18f5a97f8..24eaf0ced 100644 --- a/unit_tests/sources/declarative/test_concurrent_declarative_source.py +++ b/unit_tests/sources/declarative/test_concurrent_declarative_source.py @@ -650,14 +650,15 @@ def test_group_streams(): ) concurrent_streams, synchronous_streams = source._group_streams(config=_CONFIG) - # 1 full refresh stream, 2 incremental streams, 1 substream w/o incremental, 1 list based substream w/o incremental - assert len(concurrent_streams) == 5 + # 1 full refresh stream, 2 incremental streams, 1 substream w/o incremental, 1 list based substream w/o incremental, 1 substream w/ incremental + assert len(concurrent_streams) == 6 ( concurrent_stream_0, concurrent_stream_1, concurrent_stream_2, concurrent_stream_3, concurrent_stream_4, + concurrent_stream_5, ) = concurrent_streams assert isinstance(concurrent_stream_0, DefaultStream) assert concurrent_stream_0.name == "party_members" @@ -669,13 +670,13 @@ def test_group_streams(): assert concurrent_stream_3.name == "party_members_skills" assert isinstance(concurrent_stream_4, DefaultStream) assert concurrent_stream_4.name == "arcana_personas" + assert isinstance(concurrent_stream_5, DefaultStream) + assert concurrent_stream_5.name == "palace_enemies" - # 1 substream w/ incremental, 1 stream with async retriever - assert len(synchronous_streams) == 2 + # 1 stream with async retriever + assert len(synchronous_streams) == 1 assert isinstance(synchronous_streams[0], DeclarativeStream) - assert synchronous_streams[0].name == "palace_enemies" - assert isinstance(synchronous_streams[1], DeclarativeStream) - assert synchronous_streams[1].name == "async_job_stream" + assert synchronous_streams[0].name == "async_job_stream" @freezegun.freeze_time(time_to_freeze=datetime(2024, 9, 1, 0, 0, 0, 0, tzinfo=timezone.utc)) @@ -1456,13 +1457,13 @@ def test_streams_with_stream_state_interpolation_should_be_synchronous(): ) concurrent_streams, synchronous_streams = source._group_streams(config=_CONFIG) - # 1 full refresh stream, 2 with parent stream without incremental dependency - assert len(concurrent_streams) == 3 - # 2 incremental stream with interpolation on state (locations and party_members), 1 incremental with parent stream (palace_enemies), 1 stream with async retriever - assert len(synchronous_streams) == 4 + # 1 full refresh stream, 2 with parent stream without incremental dependency, 1 incremental with parent stream (palace_enemies) + assert len(concurrent_streams) == 4 + # 2 incremental stream with interpolation on state (locations and party_members), 1 stream with async retriever + assert len(synchronous_streams) == 3 -def test_given_partition_routing_and_incremental_sync_then_stream_is_not_concurrent(): +def test_given_partition_routing_and_incremental_sync_then_stream_is_concurrent(): manifest = { "version": "5.0.0", "definitions": { @@ -1597,8 +1598,8 @@ def test_given_partition_routing_and_incremental_sync_then_stream_is_not_concurr ) concurrent_streams, synchronous_streams = source._group_streams(config=_CONFIG) - assert len(concurrent_streams) == 0 - assert len(synchronous_streams) == 1 + assert len(concurrent_streams) == 1 + assert len(synchronous_streams) == 0 def create_wrapped_stream(stream: DeclarativeStream) -> Stream: From 23d3059956a2bac597cb9b41f36f859852fcb3ad Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Mon, 13 Jan 2025 20:11:12 +0200 Subject: [PATCH 15/34] Add request_option_provider to _request_headers of SimpleRetriever --- airbyte_cdk/sources/declarative/retrievers/simple_retriever.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py b/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py index d167a84bc..45533ac4b 100644 --- a/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py +++ b/airbyte_cdk/sources/declarative/retrievers/simple_retriever.py @@ -160,7 +160,7 @@ def _request_headers( stream_slice, next_page_token, self._paginator.get_request_headers, - self.stream_slicer.get_request_headers, + self.request_option_provider.get_request_headers, ) if isinstance(headers, str): raise ValueError("Request headers cannot be a string") From d6bec3560fb53c6a571330b38f1d5bf3a8b9cd41 Mon Sep 17 00:00:00 2001 From: Maxime Carbonneau-Leclerc <3360483+maxi297@users.noreply.github.com> Date: Tue, 14 Jan 2025 05:25:31 -0500 Subject: [PATCH 16/34] Maxi297/fix simple retriever request headers (#217) --- .../retrievers/test_simple_retriever.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/unit_tests/sources/declarative/retrievers/test_simple_retriever.py b/unit_tests/sources/declarative/retrievers/test_simple_retriever.py index 5878c758f..b33febcaf 100644 --- a/unit_tests/sources/declarative/retrievers/test_simple_retriever.py +++ b/unit_tests/sources/declarative/retrievers/test_simple_retriever.py @@ -491,8 +491,8 @@ def test_get_request_headers(test_name, paginator_mapping, expected_mapping): paginator.get_request_headers.return_value = paginator_mapping requester = MagicMock(use_cache=False) - stream_slicer = MagicMock() - stream_slicer.get_request_headers.return_value = {"key": "value"} + request_option_provider = MagicMock() + request_option_provider.get_request_headers.return_value = {"key": "value"} record_selector = MagicMock() retriever = SimpleRetriever( @@ -500,7 +500,7 @@ def test_get_request_headers(test_name, paginator_mapping, expected_mapping): primary_key=primary_key, requester=requester, record_selector=record_selector, - stream_slicer=stream_slicer, + request_option_provider=request_option_provider, paginator=paginator, parameters={}, config={}, @@ -555,7 +555,7 @@ def test_get_request_headers(test_name, paginator_mapping, expected_mapping): ), ], ) -def test_ignore_stream_slicer_parameters_on_paginated_requests( +def test_ignore_request_option_provider_parameters_on_paginated_requests( test_name, paginator_mapping, ignore_stream_slicer_parameters_on_paginated_requests, @@ -567,8 +567,8 @@ def test_ignore_stream_slicer_parameters_on_paginated_requests( paginator.get_request_headers.return_value = paginator_mapping requester = MagicMock(use_cache=False) - stream_slicer = MagicMock() - stream_slicer.get_request_headers.return_value = {"key_from_slicer": "value"} + request_option_provider = MagicMock() + request_option_provider.get_request_headers.return_value = {"key_from_slicer": "value"} record_selector = MagicMock() retriever = SimpleRetriever( @@ -576,7 +576,7 @@ def test_ignore_stream_slicer_parameters_on_paginated_requests( primary_key=primary_key, requester=requester, record_selector=record_selector, - stream_slicer=stream_slicer, + request_option_provider=request_option_provider, paginator=paginator, ignore_stream_slicer_parameters_on_paginated_requests=ignore_stream_slicer_parameters_on_paginated_requests, parameters={}, From 871f1fe9ac45434617da8336322929e59b353865 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Tue, 14 Jan 2025 12:36:53 +0200 Subject: [PATCH 17/34] Fix format --- .../declarative/incremental/__init__.py | 7 +- .../concurrent_partition_cursor.py | 9 +- .../test_concurrent_perpartitioncursor.py | 87 +++++++++++-------- 3 files changed, 56 insertions(+), 47 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/__init__.py b/airbyte_cdk/sources/declarative/incremental/__init__.py index 9852320b6..395daca6d 100644 --- a/airbyte_cdk/sources/declarative/incremental/__init__.py +++ b/airbyte_cdk/sources/declarative/incremental/__init__.py @@ -2,7 +2,10 @@ # Copyright (c) 2022 Airbyte, Inc., all rights reserved. # -from airbyte_cdk.sources.declarative.incremental.concurrent_partition_cursor import ConcurrentCursorFactory, ConcurrentPerPartitionCursor +from airbyte_cdk.sources.declarative.incremental.concurrent_partition_cursor import ( + ConcurrentCursorFactory, + ConcurrentPerPartitionCursor, +) from airbyte_cdk.sources.declarative.incremental.datetime_based_cursor import DatetimeBasedCursor from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( @@ -22,7 +25,7 @@ __all__ = [ "CursorFactory", - "ConcurrentCursorFactory" + "ConcurrentCursorFactory", "ConcurrentPerPartitionCursor", "DatetimeBasedCursor", "DeclarativeCursor", diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 83b774fe5..3efd99c1b 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -191,11 +191,7 @@ def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[Str cursor = self._cursor_per_partition.get(self._to_partition_key(partition.partition)) if not cursor: - partition_state = ( - self._global_cursor - if self._global_cursor - else self._NO_CURSOR_STATE - ) + partition_state = self._global_cursor if self._global_cursor else self._NO_CURSOR_STATE cursor = self._create_cursor(partition_state) self._cursor_per_partition[self._to_partition_key(partition.partition)] = cursor self._semaphore_per_partition[self._to_partition_key(partition.partition)] = ( @@ -286,9 +282,6 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._partition_router.set_initial_state(stream_state) def observe(self, record: Record) -> None: - print("Observing record in concirrent perpartition ", self._to_partition_key(record.associated_slice.partition), record, self._cursor_per_partition[ - self._to_partition_key(record.associated_slice.partition) - ].state) self._cursor_per_partition[ self._to_partition_key(record.associated_slice.partition) ].observe(record) diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index 989ff3158..1eafed7f4 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -243,13 +243,12 @@ SUBSTREAM_MANIFEST_NO_DEPENDENCY = deepcopy(SUBSTREAM_MANIFEST) # Disable incremental_dependency -SUBSTREAM_MANIFEST_NO_DEPENDENCY["definitions"]["post_comments_stream"]["retriever"]["partition_router"][ - "parent_stream_configs" -][0]["incremental_dependency"] = False -SUBSTREAM_MANIFEST_NO_DEPENDENCY["definitions"]["post_comment_votes_stream"]["retriever"]["partition_router"][ - "parent_stream_configs" -][0]["incremental_dependency"] = False - +SUBSTREAM_MANIFEST_NO_DEPENDENCY["definitions"]["post_comments_stream"]["retriever"][ + "partition_router" +]["parent_stream_configs"][0]["incremental_dependency"] = False +SUBSTREAM_MANIFEST_NO_DEPENDENCY["definitions"]["post_comment_votes_stream"]["retriever"][ + "partition_router" +]["parent_stream_configs"][0]["incremental_dependency"] = False def _run_read( @@ -1518,18 +1517,22 @@ def test_incremental_parent_state_no_slices( }, "cursor": {"created_at": "2024-01-03T00:00:00Z"}, }, - {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, - 'partition': {'id': 12, - 'parent_slice': {'id': 1, 'parent_slice': {}}}}, - {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, - 'partition': {'id': 20, - 'parent_slice': {'id': 2, 'parent_slice': {}}}}, - {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, - 'partition': {'id': 21, - 'parent_slice': {'id': 2, 'parent_slice': {}}}}, - {'cursor': {'created_at': '2024-01-03T00:00:00Z'}, - 'partition': {'id': 30, - 'parent_slice': {'id': 3, 'parent_slice': {}}}} + { + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + }, + { + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, + }, + { + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, + }, + { + "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, + }, ], "parent_state": { "post_comments": { @@ -1937,7 +1940,7 @@ def test_incremental_error__parent_state( }, "incremental_sync": { "$ref": "#/definitions/cursor_incremental_sync", - "is_client_side_incremental": True + "is_client_side_incremental": True, }, "$parameters": { "name": "post_comments", @@ -1958,6 +1961,7 @@ def test_incremental_error__parent_state( }, } + @pytest.mark.parametrize( "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", [ @@ -2022,28 +2026,37 @@ def test_incremental_error__parent_state( ), stream_state=AirbyteStateBlob( { - 'state': {'updated_at': '2024-01-10T00:00:00Z'}, - 'states': [{'cursor': {'updated_at': '2024-01-25T00:00:00Z'}, - 'partition': {'id': '1'}}, - {'cursor': {'updated_at': '2024-01-22T00:00:00Z'}, - 'partition': {'id': '2'}}, - {'cursor': {'updated_at': '2024-01-09T00:00:00Z'}, - 'partition': {'id': '3'}}], - 'use_global_cursor': False} + "state": {"updated_at": "2024-01-10T00:00:00Z"}, + "states": [ + { + "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + "partition": {"id": "1"}, + }, + { + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "partition": {"id": "2"}, + }, + { + "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + "partition": {"id": "3"}, + }, + ], + "use_global_cursor": False, + } ), ), ) ], # Expected state - {'lookback_window': 1, - 'state': {'updated_at': '2024-01-25T00:00:00Z'}, - 'states': [{'cursor': {'updated_at': '2024-01-25T00:00:00Z'}, - 'partition': {'id': '1'}}, - {'cursor': {'updated_at': '2024-01-22T00:00:00Z'}, - 'partition': {'id': '2'}}, - {'cursor': {'updated_at': '2024-01-09T00:00:00Z'}, - 'partition': {'id': '3'}}], - }, + { + "lookback_window": 1, + "state": {"updated_at": "2024-01-25T00:00:00Z"}, + "states": [ + {"cursor": {"updated_at": "2024-01-25T00:00:00Z"}, "partition": {"id": "1"}}, + {"cursor": {"updated_at": "2024-01-22T00:00:00Z"}, "partition": {"id": "2"}}, + {"cursor": {"updated_at": "2024-01-09T00:00:00Z"}, "partition": {"id": "3"}}, + ], + }, ), ], ) From ed687f5a418cc44b5ed604ee6adb4f7161048ab5 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Tue, 14 Jan 2025 14:02:38 +0200 Subject: [PATCH 18/34] Fix merge conflict --- .../sources/declarative/concurrent_declarative_source.py | 6 +----- .../declarative/test_concurrent_declarative_source.py | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 4639c06fb..9353c5e5e 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -339,11 +339,7 @@ def _group_streams( DeclarativePartitionFactory( declarative_stream.name, declarative_stream.get_json_schema(), - self._retriever_factory( - name_to_stream_mapping[declarative_stream.name], - config, - stream_state, - ), + declarative_stream.retriever, self.message_repository, ), cursor, diff --git a/unit_tests/sources/declarative/test_concurrent_declarative_source.py b/unit_tests/sources/declarative/test_concurrent_declarative_source.py index b0ea8a7a2..50cfb9569 100644 --- a/unit_tests/sources/declarative/test_concurrent_declarative_source.py +++ b/unit_tests/sources/declarative/test_concurrent_declarative_source.py @@ -673,9 +673,9 @@ def test_group_streams(): assert isinstance(concurrent_stream_4, DefaultStream) assert concurrent_stream_4.name == "arcana_personas" assert isinstance(concurrent_stream_5, DefaultStream) - assert concurrent_stream_5.name == "async_job_stream" + assert concurrent_stream_5.name == "palace_enemies" assert isinstance(concurrent_stream_6, DefaultStream) - assert concurrent_stream_6.name == "palace_enemies" + assert concurrent_stream_6.name == "async_job_stream" @freezegun.freeze_time(time_to_freeze=datetime(2024, 9, 1, 0, 0, 0, 0, tzinfo=timezone.utc)) From cfef872df66641521cef7ea2a12526ccdd039de0 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Tue, 14 Jan 2025 15:32:45 +0200 Subject: [PATCH 19/34] Add lookback window handling --- .../concurrent_declarative_source.py | 18 ++++++++++++++- .../concurrent_partition_cursor.py | 23 +++++++++++++------ .../parsers/model_to_component_factory.py | 6 +++++ .../test_concurrent_perpartitioncursor.py | 10 ++++---- 4 files changed, 45 insertions(+), 12 deletions(-) diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 9353c5e5e..73a148bad 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -335,11 +335,27 @@ def _group_streams( partition_router=partition_router, ) + retriever = declarative_stream.retriever + + # This is an optimization so that we don't invoke any cursor or state management flows within the + # low-code framework because state management is handled through the ConcurrentCursor. + if declarative_stream and isinstance(retriever, SimpleRetriever): + # Also a temporary hack. In the legacy Stream implementation, as part of the read, + # set_initial_state() is called to instantiate incoming state on the cursor. Although we no + # longer rely on the legacy low-code cursor for concurrent checkpointing, low-code components + # like StopConditionPaginationStrategyDecorator and ClientSideIncrementalRecordFilterDecorator + # still rely on a DatetimeBasedCursor that is properly initialized with state. + if retriever.cursor: + retriever.cursor.set_initial_state(stream_state=stream_state) + # We zero it out here, but since this is a cursor reference, the state is still properly + # instantiated for the other components that reference it + retriever.cursor = None + partition_generator = StreamSlicerPartitionGenerator( DeclarativePartitionFactory( declarative_stream.name, declarative_stream.get_json_schema(), - declarative_stream.retriever, + retriever, self.message_repository, ), cursor, diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 3efd99c1b..33f52d91f 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -7,6 +7,7 @@ import threading from collections import OrderedDict from copy import deepcopy +from datetime import timedelta from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager @@ -31,8 +32,10 @@ class ConcurrentCursorFactory: def __init__(self, create_function: Callable[..., Cursor]): self._create_function = create_function - def create(self, stream_state: Mapping[str, Any]) -> Cursor: - return self._create_function(stream_state=stream_state) + def create(self, stream_state: Mapping[str, Any], runtime_lookback_window: Any) -> Cursor: + return self._create_function( + stream_state=stream_state, runtime_lookback_window=runtime_lookback_window + ) class ConcurrentPerPartitionCursor(Cursor): @@ -125,7 +128,6 @@ def state(self) -> MutableMapping[str, Any]: return state def close_partition(self, partition: Partition) -> None: - print(f"Closing partition {self._to_partition_key(partition._stream_slice.partition)}") self._cursor_per_partition[ self._to_partition_key(partition._stream_slice.partition) ].close_partition(partition=partition) @@ -139,7 +141,6 @@ def close_partition(self, partition: Partition) -> None: cursor_state = cursor._connector_state_converter.convert_to_state_message( cursor._cursor_field, cursor.state ) - print(f"State {cursor_state} {cursor.state}") if ( self._to_partition_key(partition._stream_slice.partition) in self._finished_partitions @@ -265,9 +266,13 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._new_global_cursor = deepcopy(stream_state) else: + self._lookback_window = stream_state.get("lookback_window") + for state in stream_state["states"]: self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( - self._create_cursor(state["cursor"]) + self._create_cursor( + state["cursor"], runtime_lookback_window=self._lookback_window + ) ) self._semaphore_per_partition[self._to_partition_key(state["partition"])] = ( threading.Semaphore(0) @@ -292,8 +297,12 @@ def _to_partition_key(self, partition: Mapping[str, Any]) -> str: def _to_dict(self, partition_key: str) -> Mapping[str, Any]: return self._partition_serializer.to_partition(partition_key) - def _create_cursor(self, cursor_state: Any) -> Cursor: - cursor = self._cursor_factory.create(stream_state=deepcopy(cursor_state)) + def _create_cursor(self, cursor_state: Any, runtime_lookback_window: Any = None) -> Cursor: + if runtime_lookback_window: + runtime_lookback_window = timedelta(seconds=runtime_lookback_window) + cursor = self._cursor_factory.create( + stream_state=deepcopy(cursor_state), runtime_lookback_window=runtime_lookback_window + ) return cursor def should_be_synced(self, record: Record) -> bool: 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 7a76ea3c9..8be421d84 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -875,6 +875,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( config: Config, stream_state: MutableMapping[str, Any], message_repository: Optional[MessageRepository] = None, + runtime_lookback_window: Optional[int] = None, **kwargs: Any, ) -> ConcurrentCursor: component_type = component_definition.get("type") @@ -932,6 +933,11 @@ def create_concurrent_cursor_from_datetime_based_cursor( if evaluated_lookback_window: lookback_window = parse_duration(evaluated_lookback_window) + if runtime_lookback_window and lookback_window: + lookback_window = max(lookback_window, runtime_lookback_window) + elif runtime_lookback_window: + lookback_window = runtime_lookback_window + connector_state_converter: DateTimeStreamStateConverter connector_state_converter = CustomFormatConcurrentStreamStateConverter( datetime_format=datetime_format, diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index 1eafed7f4..cd8516902 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -320,7 +320,7 @@ def _run_read( ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-01T00:00:01Z", { "votes": [ {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} @@ -339,7 +339,7 @@ def _run_read( ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-02T00:00:00Z", { "votes": [ {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} @@ -449,6 +449,7 @@ def _run_read( "cursor": {"created_at": "2024-01-03T00:00:00Z"}, }, ], + "lookback_window": 86400, } ), ), @@ -673,7 +674,7 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-01T00:00:01Z", { "votes": [ {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} @@ -692,7 +693,7 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-02T00:00:00Z", { "votes": [ {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} @@ -844,6 +845,7 @@ def run_incremental_parent_state_test( "cursor": {"created_at": "2024-01-03T00:00:00Z"}, }, ], + "lookback_window": 86400, } ), ), From 4260415f0b3baafa407f15264e4afb2318c36e06 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Tue, 14 Jan 2025 18:03:29 +0200 Subject: [PATCH 20/34] Fix state handling in concurrent cursor --- .../concurrent_partition_cursor.py | 14 ++--- .../incremental/per_partition_cursor.py | 3 +- .../sources/streams/concurrent/cursor.py | 51 +++++++++++-------- 3 files changed, 36 insertions(+), 32 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 33f52d91f..575e9ccbf 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -107,14 +107,11 @@ def cursor_field(self) -> CursorField: def state(self) -> MutableMapping[str, Any]: states = [] for partition_tuple, cursor in self._cursor_per_partition.items(): - cursor_state = cursor._connector_state_converter.convert_to_state_message( - self.cursor_field, cursor.state - ) - if cursor_state: + if cursor.state: states.append( { "partition": self._to_dict(partition_tuple), - "cursor": copy.deepcopy(cursor_state), + "cursor": copy.deepcopy(cursor.state), } ) state: dict[str, Any] = {"states": states} @@ -138,9 +135,6 @@ def close_partition(self, partition: Partition) -> None: cursor = self._cursor_per_partition[ self._to_partition_key(partition._stream_slice.partition) ] - cursor_state = cursor._connector_state_converter.convert_to_state_message( - cursor._cursor_field, cursor.state - ) if ( self._to_partition_key(partition._stream_slice.partition) in self._finished_partitions @@ -152,9 +146,9 @@ def close_partition(self, partition: Partition) -> None: if ( self._new_global_cursor is None or self._new_global_cursor[self.cursor_field.cursor_field_key] - < cursor_state[self.cursor_field.cursor_field_key] + < cursor.state[self.cursor_field.cursor_field_key] ): - self._new_global_cursor = copy.deepcopy(cursor_state) + self._new_global_cursor = copy.deepcopy(cursor.state) def ensure_at_least_one_state_emitted(self) -> None: """ diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py index 1529e90ec..4c1eacce5 100644 --- a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py @@ -304,7 +304,8 @@ def get_request_body_json( def should_be_synced(self, record: Record) -> bool: if ( - self._to_partition_key(record.associated_slice.partition) + record.associated_slice + and self._to_partition_key(record.associated_slice.partition) not in self._cursor_per_partition ): partition_state = ( diff --git a/airbyte_cdk/sources/streams/concurrent/cursor.py b/airbyte_cdk/sources/streams/concurrent/cursor.py index cbce82a94..f453671a8 100644 --- a/airbyte_cdk/sources/streams/concurrent/cursor.py +++ b/airbyte_cdk/sources/streams/concurrent/cursor.py @@ -196,7 +196,9 @@ def __init__( @property def state(self) -> MutableMapping[str, Any]: - return self._concurrent_state + return self._connector_state_converter.convert_to_state_message( + self.cursor_field, self._concurrent_state + ) @property def cursor_field(self) -> CursorField: @@ -241,10 +243,10 @@ def _extract_cursor_value(self, record: Record) -> Any: return self._connector_state_converter.parse_value(self._cursor_field.extract_value(record)) def close_partition(self, partition: Partition) -> None: - slice_count_before = len(self.state.get("slices", [])) + slice_count_before = len(self._concurrent_state.get("slices", [])) self._add_slice_to_state(partition) if slice_count_before < len( - self.state["slices"] + self._concurrent_state["slices"] ): # only emit if at least one slice has been processed self._merge_partitions() self._emit_state_message() @@ -256,11 +258,11 @@ def _add_slice_to_state(self, partition: Partition) -> None: ) if self._slice_boundary_fields: - if "slices" not in self.state: + if "slices" not in self._concurrent_state: raise RuntimeError( f"The state for stream {self._stream_name} should have at least one slice to delineate the sync start time, but no slices are present. This is unexpected. Please contact Support." ) - self.state["slices"].append( + self._concurrent_state["slices"].append( { self._connector_state_converter.START_KEY: self._extract_from_slice( partition, self._slice_boundary_fields[self._START_BOUNDARY] @@ -288,7 +290,7 @@ def _add_slice_to_state(self, partition: Partition) -> None: "expected. Please contact the Airbyte team." ) - self.state["slices"].append( + self._concurrent_state["slices"].append( { self._connector_state_converter.START_KEY: self.start, self._connector_state_converter.END_KEY: most_recent_cursor_value, @@ -300,9 +302,7 @@ def _emit_state_message(self) -> None: self._connector_state_manager.update_state_for_stream( self._stream_name, self._stream_namespace, - self._connector_state_converter.convert_to_state_message( - self._cursor_field, self.state - ), + self.state, ) state_message = self._connector_state_manager.create_state_message( self._stream_name, self._stream_namespace @@ -310,7 +310,9 @@ def _emit_state_message(self) -> None: self._message_repository.emit_message(state_message) def _merge_partitions(self) -> None: - self.state["slices"] = self._connector_state_converter.merge_intervals(self.state["slices"]) + self._concurrent_state["slices"] = self._connector_state_converter.merge_intervals( + self._concurrent_state["slices"] + ) def _extract_from_slice(self, partition: Partition, key: str) -> CursorValueType: try: @@ -347,36 +349,42 @@ def stream_slices(self) -> Iterable[StreamSlice]: if self._start is not None and self._is_start_before_first_slice(): yield from self._split_per_slice_range( self._start, - self.state["slices"][0][self._connector_state_converter.START_KEY], + self._concurrent_state["slices"][0][self._connector_state_converter.START_KEY], False, ) - if len(self.state["slices"]) == 1: + if len(self._concurrent_state["slices"]) == 1: yield from self._split_per_slice_range( self._calculate_lower_boundary_of_last_slice( - self.state["slices"][0][self._connector_state_converter.END_KEY] + self._concurrent_state["slices"][0][self._connector_state_converter.END_KEY] ), self._end_provider(), True, ) - elif len(self.state["slices"]) > 1: - for i in range(len(self.state["slices"]) - 1): + elif len(self._concurrent_state["slices"]) > 1: + for i in range(len(self._concurrent_state["slices"]) - 1): if self._cursor_granularity: yield from self._split_per_slice_range( - self.state["slices"][i][self._connector_state_converter.END_KEY] + self._concurrent_state["slices"][i][self._connector_state_converter.END_KEY] + self._cursor_granularity, - self.state["slices"][i + 1][self._connector_state_converter.START_KEY], + self._concurrent_state["slices"][i + 1][ + self._connector_state_converter.START_KEY + ], False, ) else: yield from self._split_per_slice_range( - self.state["slices"][i][self._connector_state_converter.END_KEY], - self.state["slices"][i + 1][self._connector_state_converter.START_KEY], + self._concurrent_state["slices"][i][ + self._connector_state_converter.END_KEY + ], + self._concurrent_state["slices"][i + 1][ + self._connector_state_converter.START_KEY + ], False, ) yield from self._split_per_slice_range( self._calculate_lower_boundary_of_last_slice( - self.state["slices"][-1][self._connector_state_converter.END_KEY] + self._concurrent_state["slices"][-1][self._connector_state_converter.END_KEY] ), self._end_provider(), True, @@ -387,7 +395,8 @@ def stream_slices(self) -> Iterable[StreamSlice]: def _is_start_before_first_slice(self) -> bool: return ( self._start is not None - and self._start < self.state["slices"][0][self._connector_state_converter.START_KEY] + and self._start + < self._concurrent_state["slices"][0][self._connector_state_converter.START_KEY] ) def _calculate_lower_boundary_of_last_slice( From 089137f84b12dd89df68a07e8571bd86392458d0 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Tue, 14 Jan 2025 18:10:20 +0200 Subject: [PATCH 21/34] Fix unit test --- .../sources/declarative/test_concurrent_declarative_source.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/unit_tests/sources/declarative/test_concurrent_declarative_source.py b/unit_tests/sources/declarative/test_concurrent_declarative_source.py index 50cfb9569..ce88804c4 100644 --- a/unit_tests/sources/declarative/test_concurrent_declarative_source.py +++ b/unit_tests/sources/declarative/test_concurrent_declarative_source.py @@ -738,7 +738,7 @@ def test_create_concurrent_cursor(): assert locations_cursor._slice_range == isodate.Duration(months=1) assert locations_cursor._lookback_window == timedelta(days=5) assert locations_cursor._cursor_granularity == timedelta(days=1) - assert locations_cursor.state == { + assert locations_cursor._concurrent_state == { "slices": [ { "start": datetime(2024, 7, 1, 0, 0, 0, 0, tzinfo=timezone.utc), From 3489c7a0df5b00273443cefcb7a6cb315f27cb03 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 15 Jan 2025 15:08:07 +0200 Subject: [PATCH 22/34] Fix mypy errors --- .../concurrent_declarative_source.py | 26 +++++---- .../concurrent_partition_cursor.py | 57 +++++++------------ .../parsers/model_to_component_factory.py | 2 +- 3 files changed, 36 insertions(+), 49 deletions(-) diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 73a148bad..97322b333 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -324,15 +324,17 @@ def _group_streams( ) partition_router = declarative_stream.retriever.stream_slicer._partition_router - cursor = self._constructor.create_concurrent_cursor_from_perpartition_cursor( - state_manager=state_manager, - model_type=DatetimeBasedCursorModel, - component_definition=incremental_sync_component_definition, - stream_name=declarative_stream.name, - stream_namespace=declarative_stream.namespace, - config=config or {}, - stream_state=stream_state, - partition_router=partition_router, + perpartition_cursor = ( + self._constructor.create_concurrent_cursor_from_perpartition_cursor( + state_manager=state_manager, + model_type=DatetimeBasedCursorModel, + component_definition=incremental_sync_component_definition, + stream_name=declarative_stream.name, + stream_namespace=declarative_stream.namespace, + config=config or {}, + stream_state=stream_state, + partition_router=partition_router, + ) ) retriever = declarative_stream.retriever @@ -358,7 +360,7 @@ def _group_streams( retriever, self.message_repository, ), - cursor, + perpartition_cursor, ) concurrent_streams.append( @@ -368,9 +370,9 @@ def _group_streams( json_schema=declarative_stream.get_json_schema(), availability_strategy=AlwaysAvailableAvailabilityStrategy(), primary_key=get_primary_key_from_stream(declarative_stream.primary_key), - cursor_field=cursor.cursor_field.cursor_field_key, + cursor_field=perpartition_cursor.cursor_field.cursor_field_key, logger=self.logger, - cursor=cursor, + cursor=perpartition_cursor, ) ) else: diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 575e9ccbf..0be358d6a 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -11,7 +11,6 @@ from typing import Any, Callable, Iterable, Mapping, MutableMapping, Optional from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager -from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( Timer, iterate_with_last_flag_and_state, @@ -40,20 +39,16 @@ def create(self, stream_state: Mapping[str, Any], runtime_lookback_window: Any) class ConcurrentPerPartitionCursor(Cursor): """ - Manages state per partition when a stream has many partitions, to prevent data loss or duplication. + Manages state per partition when a stream has many partitions, preventing data loss or duplication. - **Partition Limitation and Limit Reached Logic** + Attributes: + DEFAULT_MAX_PARTITIONS_NUMBER (int): Maximum number of partitions to retain in memory (default is 10,000). - - **DEFAULT_MAX_PARTITIONS_NUMBER**: The maximum number of partitions to keep in memory (default is 10,000). - - **_cursor_per_partition**: An ordered dictionary that stores cursors for each partition. - - **_over_limit**: A counter that increments each time an oldest partition is removed when the limit is exceeded. + - **Partition Limitation Logic** + Ensures the number of tracked partitions does not exceed the specified limit to prevent memory overuse. Oldest partitions are removed when the limit is reached. - The class ensures that the number of partitions tracked does not exceed the `DEFAULT_MAX_PARTITIONS_NUMBER` to prevent excessive memory usage. - - - When the number of partitions exceeds the limit, the oldest partitions are removed from `_cursor_per_partition`, and `_over_limit` is incremented accordingly. - - The `limit_reached` method returns `True` when `_over_limit` exceeds `DEFAULT_MAX_PARTITIONS_NUMBER`, indicating that the global cursor should be used instead of per-partition cursors. - - This approach avoids unnecessary switching to a global cursor due to temporary spikes in partition counts, ensuring that switching is only done when a sustained high number of partitions is observed. + - **Global Cursor Fallback** + New partitions use global state as the initial state to progress the state for deleted or new partitions. The history data added after the initial sync will be missing. """ DEFAULT_MAX_PARTITIONS_NUMBER = 10000 @@ -73,7 +68,7 @@ def __init__( connector_state_manager: ConnectorStateManager, cursor_field: CursorField, ) -> None: - self._global_cursor: Mapping[str, Any] = {} + self._global_cursor: Optional[StreamState] = {} self._stream_name = stream_name self._stream_namespace = stream_namespace self._message_repository = message_repository @@ -86,15 +81,14 @@ def __init__( # The dict is ordered to ensure that once the maximum number of partitions is reached, # the oldest partitions can be efficiently removed, maintaining the most recent partitions. self._cursor_per_partition: OrderedDict[str, Cursor] = OrderedDict() - self._state = {"states": []} - self._semaphore_per_partition = OrderedDict() - self._finished_partitions = set() + self._semaphore_per_partition: OrderedDict[str, threading.Semaphore] = OrderedDict() + self._finished_partitions: set[str] = set() self._lock = threading.Lock() self._timer = Timer() - self._new_global_cursor = None - self._lookback_window = 0 - self._parent_state = None - self._over_limit = 0 + self._new_global_cursor: Optional[StreamState] = None + self._lookback_window: int = 0 + self._parent_state: Optional[StreamState] = None + self._over_limit: int = 0 self._partition_serializer = PerPartitionKeySerializer() self._set_initial_state(stream_state) @@ -125,23 +119,14 @@ def state(self) -> MutableMapping[str, Any]: return state def close_partition(self, partition: Partition) -> None: - self._cursor_per_partition[ - self._to_partition_key(partition._stream_slice.partition) - ].close_partition(partition=partition) + partition_key = self._to_partition_key(partition._stream_slice.partition) + self._cursor_per_partition[partition_key].close_partition(partition=partition) with self._lock: - self._semaphore_per_partition[ - self._to_partition_key(partition._stream_slice.partition) - ].acquire() - cursor = self._cursor_per_partition[ - self._to_partition_key(partition._stream_slice.partition) - ] + self._semaphore_per_partition[partition_key].acquire() + cursor = self._cursor_per_partition[partition_key] if ( - self._to_partition_key(partition._stream_slice.partition) - in self._finished_partitions - and self._semaphore_per_partition[ - self._to_partition_key(partition._stream_slice.partition) - ]._value - == 0 + partition_key in self._finished_partitions + and self._semaphore_per_partition[partition_key]._value == 0 ): if ( self._new_global_cursor is None @@ -260,7 +245,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._new_global_cursor = deepcopy(stream_state) else: - self._lookback_window = stream_state.get("lookback_window") + self._lookback_window = int(stream_state.get("lookback_window", 0)) for state in stream_state["states"]: self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( 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 8be421d84..7080f5ecf 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -1037,7 +1037,7 @@ def create_concurrent_cursor_from_perpartition_cursor( stream_namespace: Optional[str], config: Config, stream_state: MutableMapping[str, Any], - partition_router, + partition_router: PartitionRouter, **kwargs: Any, ) -> ConcurrentPerPartitionCursor: component_type = component_definition.get("type") From 301bd31921a7d0de5692cda4a0a5ac64a14f0791 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 15 Jan 2025 19:10:05 +0200 Subject: [PATCH 23/34] Add error test, fix mypy errors --- .../concurrent_partition_cursor.py | 42 ++--- .../test_concurrent_perpartitioncursor.py | 150 ++++++++++++++++-- 2 files changed, 147 insertions(+), 45 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 0be358d6a..7d5061f9a 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -20,7 +20,7 @@ from airbyte_cdk.sources.streams.checkpoint.per_partition_key_serializer import ( PerPartitionKeySerializer, ) -from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, CursorField +from airbyte_cdk.sources.streams.concurrent.cursor import ConcurrentCursor, Cursor, CursorField from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.types import Record, StreamSlice, StreamState @@ -28,10 +28,12 @@ class ConcurrentCursorFactory: - def __init__(self, create_function: Callable[..., Cursor]): + def __init__(self, create_function: Callable[..., ConcurrentCursor]): self._create_function = create_function - def create(self, stream_state: Mapping[str, Any], runtime_lookback_window: Any) -> Cursor: + def create( + self, stream_state: Mapping[str, Any], runtime_lookback_window: Any + ) -> ConcurrentCursor: return self._create_function( stream_state=stream_state, runtime_lookback_window=runtime_lookback_window ) @@ -80,7 +82,7 @@ def __init__( # The dict is ordered to ensure that once the maximum number of partitions is reached, # the oldest partitions can be efficiently removed, maintaining the most recent partitions. - self._cursor_per_partition: OrderedDict[str, Cursor] = OrderedDict() + self._cursor_per_partition: OrderedDict[str, ConcurrentCursor] = OrderedDict() self._semaphore_per_partition: OrderedDict[str, threading.Semaphore] = OrderedDict() self._finished_partitions: set[str] = set() self._lock = threading.Lock() @@ -119,7 +121,7 @@ def state(self) -> MutableMapping[str, Any]: return state def close_partition(self, partition: Partition) -> None: - partition_key = self._to_partition_key(partition._stream_slice.partition) + partition_key = self._to_partition_key(partition.to_slice().partition) self._cursor_per_partition[partition_key].close_partition(partition=partition) with self._lock: self._semaphore_per_partition[partition_key].acquire() @@ -276,7 +278,9 @@ def _to_partition_key(self, partition: Mapping[str, Any]) -> str: def _to_dict(self, partition_key: str) -> Mapping[str, Any]: return self._partition_serializer.to_partition(partition_key) - def _create_cursor(self, cursor_state: Any, runtime_lookback_window: Any = None) -> Cursor: + def _create_cursor( + self, cursor_state: Any, runtime_lookback_window: Any = None + ) -> ConcurrentCursor: if runtime_lookback_window: runtime_lookback_window = timedelta(seconds=runtime_lookback_window) cursor = self._cursor_factory.create( @@ -287,31 +291,7 @@ def _create_cursor(self, cursor_state: Any, runtime_lookback_window: Any = None) def should_be_synced(self, record: Record) -> bool: return self._get_cursor(record).should_be_synced(record) - def is_greater_than_or_equal(self, first: Record, second: Record) -> bool: - if not first.associated_slice or not second.associated_slice: - raise ValueError( - f"Both records should have an associated slice but got {first.associated_slice} and {second.associated_slice}" - ) - if first.associated_slice.partition != second.associated_slice.partition: - raise ValueError( - f"To compare records, partition should be the same but got {first.associated_slice.partition} and {second.associated_slice.partition}" - ) - - return self._get_cursor(first).is_greater_than_or_equal( - self._convert_record_to_cursor_record(first), - self._convert_record_to_cursor_record(second), - ) - - @staticmethod - def _convert_record_to_cursor_record(record: Record) -> Record: - return Record( - record.data, - StreamSlice(partition={}, cursor_slice=record.associated_slice.cursor_slice) - if record.associated_slice - else None, - ) - - def _get_cursor(self, record: Record) -> Cursor: + def _get_cursor(self, record: Record) -> ConcurrentCursor: if not record.associated_slice: raise ValueError( "Invalid state as stream slices that are emitted should refer to an existing cursor" diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index cd8516902..0041ff2fd 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -270,11 +270,17 @@ def _run_read( ) ] ) - logger = MagicMock() source = ConcurrentDeclarativeSource( source_config=manifest, config=config, catalog=catalog, state=state ) - messages = list(source.read(logger=source.logger, config=config, catalog=catalog, state=[])) + messages = [] + try: + for message in source.read( + logger=source.logger, config=config, catalog=catalog, state=state + ): + messages.append(message) + except Exception: + pass # Ignore exceptions in tests return messages @@ -2013,9 +2019,7 @@ def test_incremental_error__parent_state( [ {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, - {"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}, {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"}, - {"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}, {"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}, ], # Initial state @@ -2023,25 +2027,19 @@ def test_incremental_error__parent_state( AirbyteStateMessage( type=AirbyteStateType.STREAM, stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), + stream_descriptor=StreamDescriptor(name="post_comments", namespace=None), stream_state=AirbyteStateBlob( { - "state": {"updated_at": "2024-01-10T00:00:00Z"}, + "state": {"updated_at": "2024-01-08T00:00:00Z"}, "states": [ { - "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + "cursor": {"updated_at": "2024-01-24T00:00:00Z"}, "partition": {"id": "1"}, }, { - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "cursor": {"updated_at": "2024-01-21T05:00:00Z"}, "partition": {"id": "2"}, }, - { - "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, - "partition": {"id": "3"}, - }, ], "use_global_cursor": False, } @@ -2091,3 +2089,127 @@ def test_incremental_list_partition_router( if message.state ] assert final_state[-1] == expected_state + + +@pytest.mark.parametrize( + "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + [ + ( + "test_incremental_error_handling", + LISTPARTITION_MANIFEST, + [ + # Fetch the first page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100", + { + "comments": [ + {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + ], + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + }, + ), + # Error response for the second page of comments for post 1 + ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + None, # Simulate a network error or an empty response + ), + # Fetch the first page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100", + { + "comments": [ + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + ], + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + }, + ), + # Fetch the second page of comments for post 2 + ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + ), + # Fetch the first page of comments for post 3 + ( + "https://api.example.com/community/posts/3/comments?per_page=100", + {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + ), + ], + # Expected records + [ + {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, + {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"}, + {"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}, + ], + # Initial state + [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(name="post_comments", namespace=None), + stream_state=AirbyteStateBlob( + { + "state": {"updated_at": "2024-01-08T00:00:00Z"}, + "states": [ + { + "cursor": {"updated_at": "2024-01-20T00:00:00Z"}, + "partition": {"id": "1"}, + }, + { + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "partition": {"id": "2"}, + }, + ], + "use_global_cursor": False, + } + ), + ), + ) + ], + # Expected state + { + "lookback_window": 0, + "state": {"updated_at": "2024-01-08T00:00:00Z"}, + "states": [ + {"cursor": {"updated_at": "2024-01-20T00:00:00Z"}, "partition": {"id": "1"}}, + {"cursor": {"updated_at": "2024-01-22T00:00:00Z"}, "partition": {"id": "2"}}, + {"cursor": {"updated_at": "2024-01-09T00:00:00Z"}, "partition": {"id": "3"}}, + ], + }, + ), + ], +) +def test_incremental_error( + test_name, manifest, mock_requests, expected_records, initial_state, expected_state +): + """ + Test with failed request. + """ + + _stream_name = "post_comments" + config = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, + } + + with requests_mock.Mocker() as m: + for url, response in mock_requests: + if response is None: + m.get(url, status_code=404) + else: + m.get(url, json=response) + + output = _run_read(manifest, config, _stream_name, initial_state) + output_data = [message.record.data for message in output if message.record] + + assert set(tuple(sorted(d.items())) for d in output_data) == set( + tuple(sorted(d.items())) for d in expected_records + ) + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] + assert final_state[-1] == expected_state From 9574f8c061eaa48512972190a5617c81b6db73a7 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 15 Jan 2025 19:29:19 +0200 Subject: [PATCH 24/34] Fix stream slice mypy errors --- .../incremental/concurrent_partition_cursor.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 7d5061f9a..90345363f 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -121,7 +121,14 @@ def state(self) -> MutableMapping[str, Any]: return state def close_partition(self, partition: Partition) -> None: - partition_key = self._to_partition_key(partition.to_slice().partition) + # Attempt to retrieve the stream slice + stream_slice: Optional[StreamSlice] = partition.to_slice() # type: ignore[assignment] + + # Ensure stream_slice is not None + if stream_slice is None: + raise ValueError("stream_slice cannot be None") + + partition_key = self._to_partition_key(stream_slice.partition) self._cursor_per_partition[partition_key].close_partition(partition=partition) with self._lock: self._semaphore_per_partition[partition_key].acquire() @@ -268,6 +275,10 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._partition_router.set_initial_state(stream_state) def observe(self, record: Record) -> None: + if not record.associated_slice: + raise ValueError( + "Invalid state as stream slices that are emitted should refer to an existing cursor" + ) self._cursor_per_partition[ self._to_partition_key(record.associated_slice.partition) ].observe(record) From 5ab4ee3c239d848171c390665a7a8573521eca45 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 16 Jan 2025 14:00:21 +0200 Subject: [PATCH 25/34] Fix lookback window --- .../incremental/concurrent_partition_cursor.py | 10 +++++----- .../incremental/test_concurrent_perpartitioncursor.py | 8 ++++---- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 90345363f..e8608e67f 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -180,8 +180,10 @@ def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[Str cursor = self._cursor_per_partition.get(self._to_partition_key(partition.partition)) if not cursor: - partition_state = self._global_cursor if self._global_cursor else self._NO_CURSOR_STATE - cursor = self._create_cursor(partition_state) + cursor = self._create_cursor( + self._global_cursor, + self._lookback_window if self._global_cursor else self._NO_CURSOR_STATE, + ) self._cursor_per_partition[self._to_partition_key(partition.partition)] = cursor self._semaphore_per_partition[self._to_partition_key(partition.partition)] = ( threading.Semaphore(0) @@ -258,9 +260,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: for state in stream_state["states"]: self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( - self._create_cursor( - state["cursor"], runtime_lookback_window=self._lookback_window - ) + self._create_cursor(state["cursor"]) ) self._semaphore_per_partition[self._to_partition_key(state["partition"])] = ( threading.Semaphore(0) diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index 0041ff2fd..f2fc18bcc 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -326,7 +326,7 @@ def _run_read( ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", { "votes": [ {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} @@ -345,7 +345,7 @@ def _run_read( ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", { "votes": [ {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} @@ -680,7 +680,7 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", { "votes": [ {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} @@ -699,7 +699,7 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", { "votes": [ {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} From 36c499213184c64a9703d2ef60b4ccd15a31c9b0 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 16 Jan 2025 18:25:25 +0200 Subject: [PATCH 26/34] Refactor unit tests --- .../concurrent_partition_cursor.py | 4 +- .../test_concurrent_perpartitioncursor.py | 404 +++++++++--------- 2 files changed, 201 insertions(+), 207 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index e8608e67f..712f686d0 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -172,9 +172,9 @@ def stream_slices(self) -> Iterable[StreamSlice]: slices = self._partition_router.stream_slices() self._timer.start() for partition in slices: - yield from self.generate_slices_from_partition(partition) + yield from self._generate_slices_from_partition(partition) - def generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[StreamSlice]: + def _generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[StreamSlice]: # Ensure the maximum number of partitions is not exceeded self._ensure_partition_limit() diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index f2fc18bcc..b1eecb3b8 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -3,28 +3,22 @@ import copy from copy import deepcopy from typing import Any, List, Mapping, MutableMapping, Optional, Union -from unittest.mock import MagicMock import pytest -import requests_mock from orjson import orjson from airbyte_cdk.models import ( - AirbyteMessage, AirbyteStateBlob, AirbyteStateMessage, AirbyteStateType, - AirbyteStream, AirbyteStreamState, - ConfiguredAirbyteCatalog, - ConfiguredAirbyteStream, - DestinationSyncMode, StreamDescriptor, - SyncMode, ) from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( ConcurrentDeclarativeSource, ) +from airbyte_cdk.test.catalog_builder import CatalogBuilder, ConfiguredAirbyteStreamBuilder +from airbyte_cdk.test.entrypoint_wrapper import EntrypointOutput, read SUBSTREAM_MANIFEST: MutableMapping[str, Any] = { "version": "0.51.42", @@ -239,6 +233,47 @@ "default_concurrency": "{{ config['num_workers'] or 10 }}", "max_concurrency": 25, }, + "spec": { + "type": "Spec", + "documentation_url": "https://airbyte.com/#yaml-from-manifest", + "connection_specification": { + "title": "Test Spec", + "type": "object", + "required": ["credentials", "start_date"], + "additionalProperties": False, + "properties": { + "credentials": { + "type": "object", + "required": ["email", "api_token"], + "properties": { + "email": { + "type": "string", + "title": "Email", + "description": "The email for authentication.", + }, + "api_token": { + "type": "string", + "airbyte_secret": True, + "title": "API Token", + "description": "The API token for authentication.", + }, + }, + }, + "start_date": { + "type": "string", + "format": "date-time", + "title": "Start Date", + "description": "The date from which to start syncing data.", + }, + }, + }, + }, +} + +STREAM_NAME = "post_comment_votes" +CONFIG = { + "start_date": "2024-01-01T00:00:01Z", + "credentials": {"email": "email", "api_token": "api_token"}, } SUBSTREAM_MANIFEST_NO_DEPENDENCY = deepcopy(SUBSTREAM_MANIFEST) @@ -250,38 +285,64 @@ "partition_router" ]["parent_stream_configs"][0]["incremental_dependency"] = False +import orjson +import requests_mock + + +def run_mocked_test( + mock_requests, manifest, config, stream_name, initial_state, expected_records, expected_state +): + """ + Helper function to mock requests, run the test, and verify the results. + + Args: + mock_requests (list): List of tuples containing the URL and response data to mock. + manifest (dict): Manifest configuration for the source. + config (dict): Source configuration. + stream_name (str): Name of the stream being tested. + initial_state (dict): Initial state for the stream. + expected_records (list): Expected records to be returned by the stream. + expected_state (dict): Expected state after processing the records. + + Raises: + AssertionError: If the test output does not match the expected records or state. + """ + with requests_mock.Mocker() as m: + for url, response in mock_requests: + if response is None: + m.get(url, status_code=404) + else: + m.get(url, json=response) + + output = _run_read(manifest, config, stream_name, initial_state) + + # Verify records + assert sorted( + [r.record.data for r in output.records], key=lambda x: orjson.dumps(x) + ) == sorted(expected_records, key=lambda x: orjson.dumps(x)) + + # Verify state + final_state = output.state_messages[-1].state.stream.stream_state + assert final_state == AirbyteStateBlob(expected_state) + def _run_read( manifest: Mapping[str, Any], config: Mapping[str, Any], stream_name: str, state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, -) -> List[AirbyteMessage]: - catalog = ConfiguredAirbyteCatalog( - streams=[ - ConfiguredAirbyteStream( - stream=AirbyteStream( - name=stream_name, - json_schema={}, - supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], - ), - sync_mode=SyncMode.incremental, - destination_sync_mode=DestinationSyncMode.append, - ) - ] - ) +) -> EntrypointOutput: source = ConcurrentDeclarativeSource( - source_config=manifest, config=config, catalog=catalog, state=state + source_config=manifest, config=config, catalog=None, state=state + ) + output = read( + source, + config, + CatalogBuilder() + .with_stream(ConfiguredAirbyteStreamBuilder().with_name(stream_name)) + .build(), ) - messages = [] - try: - for message in source.read( - logger=source.logger, config=config, catalog=catalog, state=state - ): - messages.append(message) - except Exception: - pass # Ignore exceptions in tests - return messages + return output @pytest.mark.parametrize( @@ -511,29 +572,15 @@ def test_incremental_parent_state_no_incremental_dependency( parent stream requests use the incoming config as query parameters and the substream state messages does not contain parent stream state. """ - - _stream_name = "post_comment_votes" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert set(tuple(sorted(d.items())) for d in output_data) == set( - tuple(sorted(d.items())) for d in expected_records - ) - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + STREAM_NAME, + initial_state, + expected_records, + expected_state, + ) def run_incremental_parent_state_test( @@ -562,6 +609,7 @@ def run_incremental_parent_state_test( "start_date": "2024-01-01T00:00:01Z", "credentials": {"email": "email", "api_token": "api_token"}, } + expected_states = [AirbyteStateBlob(s) for s in expected_states] with requests_mock.Mocker() as m: for url, response in mock_requests: @@ -569,12 +617,11 @@ def run_incremental_parent_state_test( # Run the initial read output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] # Assert that output_data equals expected_records - assert sorted(output_data, key=lambda x: orjson.dumps(x)) == sorted( - expected_records, key=lambda x: orjson.dumps(x) - ) + assert sorted( + [r.record.data for r in output.records], key=lambda x: orjson.dumps(x) + ) == sorted(expected_records, key=lambda x: orjson.dumps(x)) # Collect the intermediate states and records produced before each state cumulative_records = [] @@ -582,14 +629,9 @@ def run_incremental_parent_state_test( final_states = [] # To store the final state after each read # Store the final state after the initial read - final_state_initial = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - final_states.append(final_state_initial[-1]) + final_states.append(output.state_messages[-1].state.stream.stream_state) - for message in output: + for message in output.records_and_state_messages: if message.type.value == "RECORD": record_data = message.record.data cumulative_records.append(record_data) @@ -602,9 +644,7 @@ def run_incremental_parent_state_test( # For each intermediate state, perform another read starting from that state for state, records_before_state in intermediate_states[:-1]: output_intermediate = _run_read(manifest, config, _stream_name, [state]) - records_from_state = [ - message.record.data for message in output_intermediate if message.record - ] + records_from_state = [r.record.data for r in output_intermediate.records] # Combine records produced before the state with records from the new read cumulative_records_state = records_before_state + records_from_state @@ -626,8 +666,7 @@ def run_incremental_parent_state_test( # Store the final state after each intermediate read final_state_intermediate = [ orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output_intermediate - if message.state + for message in output_intermediate.state_messages ] final_states.append(final_state_intermediate[-1]) @@ -923,7 +962,6 @@ def test_incremental_parent_state( "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, } additional_expected_state["states"].append(empty_state) - print(manifest) run_incremental_parent_state_test( manifest, mock_requests, @@ -1137,28 +1175,15 @@ def test_incremental_parent_state_migration( """ Test incremental partition router with parent state migration """ - _stream_name = "post_comment_votes" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert set(tuple(sorted(d.items())) for d in output_data) == set( - tuple(sorted(d.items())) for d in expected_records - ) - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + STREAM_NAME, + initial_state, + expected_records, + expected_state, + ) @pytest.mark.parametrize( @@ -1336,26 +1361,15 @@ def test_incremental_parent_state_no_slices( """ Test incremental partition router with no parent records """ - _stream_name = "post_comment_votes" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert output_data == expected_records - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + STREAM_NAME, + initial_state, + expected_records, + expected_state, + ) @pytest.mark.parametrize( @@ -1574,26 +1588,15 @@ def test_incremental_parent_state_no_records( """ Test incremental partition router with no child records """ - _stream_name = "post_comment_votes" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert output_data == expected_records - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + STREAM_NAME, + initial_state, + expected_records, + expected_state, + ) @pytest.mark.parametrize( @@ -1827,28 +1830,15 @@ def test_incremental_parent_state_no_records( def test_incremental_error__parent_state( test_name, manifest, mock_requests, expected_records, initial_state, expected_state ): - _stream_name = "post_comment_votes" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert set(tuple(sorted(d.items())) for d in output_data) == set( - tuple(sorted(d.items())) for d in expected_records - ) - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + STREAM_NAME, + initial_state, + expected_records, + expected_state, + ) LISTPARTITION_MANIFEST: MutableMapping[str, Any] = { @@ -1967,6 +1957,41 @@ def test_incremental_error__parent_state( "default_concurrency": "{{ config['num_workers'] or 10 }}", "max_concurrency": 25, }, + "spec": { + "type": "Spec", + "documentation_url": "https://airbyte.com/#yaml-from-manifest", + "connection_specification": { + "title": "Test Spec", + "type": "object", + "required": ["credentials", "start_date"], + "additionalProperties": False, + "properties": { + "credentials": { + "type": "object", + "required": ["email", "api_token"], + "properties": { + "email": { + "type": "string", + "title": "Email", + "description": "The email for authentication.", + }, + "api_token": { + "type": "string", + "airbyte_secret": True, + "title": "API Token", + "description": "The API token for authentication.", + }, + }, + }, + "start_date": { + "type": "string", + "format": "date-time", + "title": "Start Date", + "description": "The date from which to start syncing data.", + }, + }, + }, + }, } @@ -2066,29 +2091,15 @@ def test_incremental_list_partition_router( """ Test ConcurrentPerPartitionCursor with ListPartitionRouter """ - - _stream_name = "post_comments" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert set(tuple(sorted(d.items())) for d in output_data) == set( - tuple(sorted(d.items())) for d in expected_records - ) - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + "post_comments", + initial_state, + expected_records, + expected_state, + ) @pytest.mark.parametrize( @@ -2187,29 +2198,12 @@ def test_incremental_error( """ Test with failed request. """ - - _stream_name = "post_comments" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } - - with requests_mock.Mocker() as m: - for url, response in mock_requests: - if response is None: - m.get(url, status_code=404) - else: - m.get(url, json=response) - - output = _run_read(manifest, config, _stream_name, initial_state) - output_data = [message.record.data for message in output if message.record] - - assert set(tuple(sorted(d.items())) for d in output_data) == set( - tuple(sorted(d.items())) for d in expected_records - ) - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == expected_state + run_mocked_test( + mock_requests, + manifest, + CONFIG, + "post_comments", + initial_state, + expected_records, + expected_state, + ) From b6707ef0a1fcaa68998ed12968a411b685b16c2b Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 16 Jan 2025 19:13:11 +0200 Subject: [PATCH 27/34] Refactor to add helper to get retriever --- .../concurrent_partition_cursor.py | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 712f686d0..7e18601ef 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -218,13 +218,19 @@ def limit_reached(self) -> bool: def _set_initial_state(self, stream_state: StreamState) -> None: """ - Set the initial state for the cursors. + Initialize the cursor's state using the provided `stream_state`. - This method initializes the state for each partition cursor using the provided stream state. - If a partition state is provided in the stream state, it will update the corresponding partition cursor with this state. + This method supports global and per-partition state initialization. - Additionally, it sets the parent state for partition routers that are based on parent streams. If a partition router - does not have parent streams, this step will be skipped due to the default PartitionRouter implementation. + - **Global State**: If `states` is missing, the `state` is treated as global and applied to all partitions. + The `global state` holds a single cursor position representing the latest processed record across all partitions. + + - **Lookback Window**: Configured via `lookback_window`, it defines the period (in seconds) for reprocessing records. + This ensures robustness in case of upstream data delays or reordering. If not specified, it defaults to 0. + + - **Per-Partition State**: If `states` is present, each partition's cursor state is initialized separately. + + - **Parent State**: (if available) Used to initialize partition routers based on parent streams. Args: stream_state (StreamState): The state of the streams to be set. The format of the stream state should be: @@ -239,6 +245,10 @@ def _set_initial_state(self, stream_state: StreamState) -> None: } } ], + "state": { + "last_updated": "2023-05-27T00:00:00Z" + }, + lookback_window: 10, "parent_state": { "parent_stream_name": { "last_updated": "2023-05-27T00:00:00Z" From cf5107f3d75db401dc1512a32be485558783684d Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 16 Jan 2025 19:19:18 +0200 Subject: [PATCH 28/34] Refactor to add helper to get retriever --- .../concurrent_declarative_source.py | 55 ++++++++----------- 1 file changed, 24 insertions(+), 31 deletions(-) diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 97322b333..63e6aacb2 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -35,7 +35,7 @@ ModelToComponentFactory, ) from airbyte_cdk.sources.declarative.requesters import HttpRequester -from airbyte_cdk.sources.declarative.retrievers import SimpleRetriever +from airbyte_cdk.sources.declarative.retrievers import Retriever, SimpleRetriever from airbyte_cdk.sources.declarative.stream_slicers.declarative_partition_generator import ( DeclarativePartitionFactory, StreamSlicerPartitionGenerator, @@ -233,21 +233,7 @@ def _group_streams( stream_state=stream_state, ) - retriever = declarative_stream.retriever - - # This is an optimization so that we don't invoke any cursor or state management flows within the - # low-code framework because state management is handled through the ConcurrentCursor. - if declarative_stream and isinstance(retriever, SimpleRetriever): - # Also a temporary hack. In the legacy Stream implementation, as part of the read, - # set_initial_state() is called to instantiate incoming state on the cursor. Although we no - # longer rely on the legacy low-code cursor for concurrent checkpointing, low-code components - # like StopConditionPaginationStrategyDecorator and ClientSideIncrementalRecordFilterDecorator - # still rely on a DatetimeBasedCursor that is properly initialized with state. - if retriever.cursor: - retriever.cursor.set_initial_state(stream_state=stream_state) - # We zero it out here, but since this is a cursor reference, the state is still properly - # instantiated for the other components that reference it - retriever.cursor = None + retriever = self._get_retriever(declarative_stream, stream_state) partition_generator = StreamSlicerPartitionGenerator( DeclarativePartitionFactory( @@ -337,21 +323,7 @@ def _group_streams( ) ) - retriever = declarative_stream.retriever - - # This is an optimization so that we don't invoke any cursor or state management flows within the - # low-code framework because state management is handled through the ConcurrentCursor. - if declarative_stream and isinstance(retriever, SimpleRetriever): - # Also a temporary hack. In the legacy Stream implementation, as part of the read, - # set_initial_state() is called to instantiate incoming state on the cursor. Although we no - # longer rely on the legacy low-code cursor for concurrent checkpointing, low-code components - # like StopConditionPaginationStrategyDecorator and ClientSideIncrementalRecordFilterDecorator - # still rely on a DatetimeBasedCursor that is properly initialized with state. - if retriever.cursor: - retriever.cursor.set_initial_state(stream_state=stream_state) - # We zero it out here, but since this is a cursor reference, the state is still properly - # instantiated for the other components that reference it - retriever.cursor = None + retriever = self._get_retriever(declarative_stream, stream_state) partition_generator = StreamSlicerPartitionGenerator( DeclarativePartitionFactory( @@ -465,6 +437,27 @@ def _stream_supports_concurrent_partition_processing( return False return True + def _get_retriever( + self, declarative_stream: DeclarativeStream, stream_state: Mapping[str, Any] + ) -> Retriever: + retriever = declarative_stream.retriever + + # This is an optimization so that we don't invoke any cursor or state management flows within the + # low-code framework because state management is handled through the ConcurrentCursor. + if declarative_stream and isinstance(retriever, SimpleRetriever): + # Also a temporary hack. In the legacy Stream implementation, as part of the read, + # set_initial_state() is called to instantiate incoming state on the cursor. Although we no + # longer rely on the legacy low-code cursor for concurrent checkpointing, low-code components + # like StopConditionPaginationStrategyDecorator and ClientSideIncrementalRecordFilterDecorator + # still rely on a DatetimeBasedCursor that is properly initialized with state. + if retriever.cursor: + retriever.cursor.set_initial_state(stream_state=stream_state) + # We zero it out here, but since this is a cursor reference, the state is still properly + # instantiated for the other components that reference it + retriever.cursor = None + + return retriever + @staticmethod def _select_streams( streams: List[AbstractStream], configured_catalog: ConfiguredAirbyteCatalog From df0993ee37bd69bbbc6dc11a5830a2c87309626f Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 16 Jan 2025 19:19:46 +0200 Subject: [PATCH 29/34] Add class variable for state keys --- .../incremental/concurrent_partition_cursor.py | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 7e18601ef..37d2eb8da 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -56,6 +56,8 @@ class ConcurrentPerPartitionCursor(Cursor): DEFAULT_MAX_PARTITIONS_NUMBER = 10000 _NO_STATE: Mapping[str, Any] = {} _NO_CURSOR_STATE: Mapping[str, Any] = {} + _GLOBAL_STATE_KEY = "state" + _PERPARTITION_STATE_KEY = "states" _KEY = 0 _VALUE = 1 @@ -110,10 +112,10 @@ def state(self) -> MutableMapping[str, Any]: "cursor": copy.deepcopy(cursor.state), } ) - state: dict[str, Any] = {"states": states} + state: dict[str, Any] = {self._PERPARTITION_STATE_KEY: states} if self._global_cursor: - state["state"] = self._global_cursor + state[self._GLOBAL_STATE_KEY] = self._global_cursor if self._lookback_window is not None: state["lookback_window"] = self._lookback_window if self._parent_state is not None: @@ -259,7 +261,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: if not stream_state: return - if "states" not in stream_state: + if self._PERPARTITION_STATE_KEY not in stream_state: # We assume that `stream_state` is in a global format that can be applied to all partitions. # Example: {"global_state_format_key": "global_state_format_value"} self._global_cursor = deepcopy(stream_state) @@ -268,7 +270,7 @@ def _set_initial_state(self, stream_state: StreamState) -> None: else: self._lookback_window = int(stream_state.get("lookback_window", 0)) - for state in stream_state["states"]: + for state in stream_state[self._PERPARTITION_STATE_KEY]: self._cursor_per_partition[self._to_partition_key(state["partition"])] = ( self._create_cursor(state["cursor"]) ) @@ -277,9 +279,9 @@ def _set_initial_state(self, stream_state: StreamState) -> None: ) # set default state for missing partitions if it is per partition with fallback to global - if "state" in stream_state: - self._global_cursor = deepcopy(stream_state["state"]) - self._new_global_cursor = deepcopy(stream_state["state"]) + if self._GLOBAL_STATE_KEY in stream_state: + self._global_cursor = deepcopy(stream_state[self._GLOBAL_STATE_KEY]) + self._new_global_cursor = deepcopy(stream_state[self._GLOBAL_STATE_KEY]) # Set parent state for partition routers based on parent streams self._partition_router.set_initial_state(stream_state) From daa6873f02983fd7d49e1576f97a3dc76c3378dc Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Thu, 16 Jan 2025 19:28:25 +0200 Subject: [PATCH 30/34] Add exception if stream_slices was executed two times --- .../incremental/concurrent_partition_cursor.py | 9 ++++++--- .../declarative/incremental/global_substream_cursor.py | 3 +++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 37d2eb8da..2e6d5d2b9 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -1,9 +1,9 @@ -import copy -import logging - # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # + +import copy +import logging import threading from collections import OrderedDict from copy import deepcopy @@ -171,6 +171,9 @@ def _emit_state_message(self) -> None: self._message_repository.emit_message(state_message) def stream_slices(self) -> Iterable[StreamSlice]: + if self._timer.is_running(): + raise RuntimeError("stream_slices has been executed more than once.") + slices = self._partition_router.stream_slices() self._timer.start() for partition in slices: diff --git a/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py b/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py index 3b3636236..f5439b9ce 100644 --- a/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py @@ -64,6 +64,9 @@ def finish(self) -> int: else: raise RuntimeError("Global substream cursor timer not started") + def is_running(self) -> bool: + return self._start is not None + class GlobalSubstreamCursor(DeclarativeCursor): """ From c827d82f4efb66523ae76ad9d2c3fad7cfdded52 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 22 Jan 2025 17:23:41 +0200 Subject: [PATCH 31/34] Fix issues with error handling, refactor tests --- .../concurrent_partition_cursor.py | 18 +- .../parsers/model_to_component_factory.py | 19 +- .../test_concurrent_perpartitioncursor.py | 1426 +++++++++-------- 3 files changed, 784 insertions(+), 679 deletions(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 2e6d5d2b9..be7167493 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -32,7 +32,7 @@ def __init__(self, create_function: Callable[..., ConcurrentCursor]): self._create_function = create_function def create( - self, stream_state: Mapping[str, Any], runtime_lookback_window: Any + self, stream_state: Mapping[str, Any], runtime_lookback_window: Optional[timedelta] ) -> ConcurrentCursor: return self._create_function( stream_state=stream_state, runtime_lookback_window=runtime_lookback_window @@ -187,7 +187,7 @@ def _generate_slices_from_partition(self, partition: StreamSlice) -> Iterable[St if not cursor: cursor = self._create_cursor( self._global_cursor, - self._lookback_window if self._global_cursor else self._NO_CURSOR_STATE, + self._lookback_window if self._global_cursor else 0, ) self._cursor_per_partition[self._to_partition_key(partition.partition)] = cursor self._semaphore_per_partition[self._to_partition_key(partition.partition)] = ( @@ -218,9 +218,6 @@ def _ensure_partition_limit(self) -> None: f"The maximum number of partitions has been reached. Dropping the oldest partition: {oldest_partition}. Over limit: {self._over_limit}." ) - def limit_reached(self) -> bool: - return self._over_limit > self.DEFAULT_MAX_PARTITIONS_NUMBER - def _set_initial_state(self, stream_state: StreamState) -> None: """ Initialize the cursor's state using the provided `stream_state`. @@ -286,6 +283,10 @@ def _set_initial_state(self, stream_state: StreamState) -> None: self._global_cursor = deepcopy(stream_state[self._GLOBAL_STATE_KEY]) self._new_global_cursor = deepcopy(stream_state[self._GLOBAL_STATE_KEY]) + # Set initial parent state + if stream_state.get("parent_state"): + self._parent_state = stream_state["parent_state"] + # Set parent state for partition routers based on parent streams self._partition_router.set_initial_state(stream_state) @@ -305,12 +306,11 @@ def _to_dict(self, partition_key: str) -> Mapping[str, Any]: return self._partition_serializer.to_partition(partition_key) def _create_cursor( - self, cursor_state: Any, runtime_lookback_window: Any = None + self, cursor_state: Any, runtime_lookback_window: int = 0 ) -> ConcurrentCursor: - if runtime_lookback_window: - runtime_lookback_window = timedelta(seconds=runtime_lookback_window) cursor = self._cursor_factory.create( - stream_state=deepcopy(cursor_state), runtime_lookback_window=runtime_lookback_window + stream_state=deepcopy(cursor_state), + runtime_lookback_window=timedelta(seconds=runtime_lookback_window), ) return cursor 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 7080f5ecf..81f3bb7ff 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -875,7 +875,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( config: Config, stream_state: MutableMapping[str, Any], message_repository: Optional[MessageRepository] = None, - runtime_lookback_window: Optional[int] = None, + runtime_lookback_window: Optional[datetime.timedelta] = None, **kwargs: Any, ) -> ConcurrentCursor: component_type = component_definition.get("type") @@ -933,11 +933,6 @@ def create_concurrent_cursor_from_datetime_based_cursor( if evaluated_lookback_window: lookback_window = parse_duration(evaluated_lookback_window) - if runtime_lookback_window and lookback_window: - lookback_window = max(lookback_window, runtime_lookback_window) - elif runtime_lookback_window: - lookback_window = runtime_lookback_window - connector_state_converter: DateTimeStreamStateConverter connector_state_converter = CustomFormatConcurrentStreamStateConverter( datetime_format=datetime_format, @@ -946,6 +941,18 @@ def create_concurrent_cursor_from_datetime_based_cursor( cursor_granularity=cursor_granularity, ) + # Adjusts the stream state by applying the runtime lookback window. + # This is used to ensure correct state handling in case of failed partitions. + stream_state_value = stream_state.get(cursor_field.cursor_field_key) + if runtime_lookback_window and stream_state_value: + new_stream_state = ( + connector_state_converter.parse_timestamp(stream_state_value) + - runtime_lookback_window + ) + stream_state[cursor_field.cursor_field_key] = connector_state_converter.output_format( + new_stream_state + ) + start_date_runtime_value: Union[InterpolatedString, str, MinMaxDatetime] if isinstance(datetime_based_cursor_model.start_datetime, MinMaxDatetimeModel): start_date_runtime_value = self.create_min_max_datetime( diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index b1eecb3b8..b0abfc6e7 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -2,6 +2,7 @@ import copy from copy import deepcopy +from datetime import datetime, timedelta from typing import Any, List, Mapping, MutableMapping, Optional, Union import pytest @@ -209,10 +210,23 @@ "parent_key": "id", "partition_field": "id", "incremental_dependency": True, + "extra_fields": [["updated_at"]], } ], }, }, + "transformations": [ + { + "type": "AddFields", + "fields": [ + { + "path": ["comment_updated_at"], + "value_type": "string", + "value": "{{ stream_slice.extra_fields['updated_at'] }}", + }, + ], + }, + ], "incremental_sync": "#/definitions/cursor_incremental_sync", "$parameters": { "name": "post_comment_votes", @@ -314,16 +328,25 @@ def run_mocked_test( else: m.get(url, json=response) + initial_state = [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(name=stream_name, namespace=None), + stream_state=AirbyteStateBlob(initial_state), + ), + ) + ] output = _run_read(manifest, config, stream_name, initial_state) # Verify records - assert sorted( - [r.record.data for r in output.records], key=lambda x: orjson.dumps(x) - ) == sorted(expected_records, key=lambda x: orjson.dumps(x)) + assert sorted([r.record.data for r in output.records], key=lambda x: x["id"]) == sorted( + expected_records, key=lambda x: x["id"] + ) # Verify state final_state = output.state_messages[-1].state.stream.stream_state - assert final_state == AirbyteStateBlob(expected_state) + assert final_state.__dict__ == expected_state def _run_read( @@ -345,6 +368,46 @@ def _run_read( return output +# Existing Constants for Dates +START_DATE = "2024-01-01T00:00:01Z" # Start of the sync +POST_1_UPDATED_AT = "2024-01-30T00:00:00Z" # Latest update date for post 1 +POST_2_UPDATED_AT = "2024-01-29T00:00:00Z" # Latest update date for post 2 +POST_3_UPDATED_AT = "2024-01-28T00:00:00Z" # Latest update date for post 3 + +COMMENT_9_OLDEST = "2023-01-01T00:00:00Z" # Comment in partition 1 - filtered out due to date +COMMENT_10_UPDATED_AT = "2024-01-25T00:00:00Z" # Latest comment in partition 1 +COMMENT_11_UPDATED_AT = "2024-01-24T00:00:00Z" # Comment in partition 1 +COMMENT_12_UPDATED_AT = "2024-01-23T00:00:00Z" # Comment in partition 1 +COMMENT_20_UPDATED_AT = "2024-01-22T00:00:00Z" # Latest comment in partition 2 +COMMENT_21_UPDATED_AT = "2024-01-21T00:00:00Z" # Comment in partition 2 +COMMENT_30_UPDATED_AT = "2024-01-09T00:00:00Z" # Latest comment in partition 3 +LOOKBACK_WINDOW_DAYS = 1 # Lookback window duration in days + +# Votes Date Constants +VOTE_100_CREATED_AT = "2024-01-15T00:00:00Z" # Latest vote in partition 1 +VOTE_101_CREATED_AT = "2024-01-14T00:00:00Z" # Second-latest vote in partition 1 +VOTE_111_CREATED_AT = "2024-01-13T00:00:00Z" # Latest vote in partition 2 +VOTE_200_CREATED_AT = "2024-01-12T00:00:00Z" # Latest vote in partition 3 +VOTE_201_CREATED_AT = "2024-01-12T00:00:15Z" # Second-latest vote in partition 3 +VOTE_300_CREATED_AT = "2024-01-10T00:00:00Z" # Latest vote in partition 4 + +# Initial State Constants +PARENT_COMMENT_CURSOR_PARTITION_1 = "2023-01-04T00:00:00Z" # Parent comment cursor (partition 1) +PARENT_POSTS_CURSOR = "2024-01-05T00:00:00Z" # Parent posts cursor (expected in state) + +INITIAL_STATE_PARTITION_10_CURSOR = "2024-01-02T00:00:01Z" +INITIAL_STATE_PARTITION_11_CURSOR = "2024-01-03T00:00:02Z" +INITIAL_GLOBAL_CURSOR = INITIAL_STATE_PARTITION_11_CURSOR +INITIAL_GLOBAL_CURSOR_DATE = datetime.fromisoformat( + INITIAL_STATE_PARTITION_11_CURSOR.replace("Z", "") +) +LOOKBACK_DATE = ( + INITIAL_GLOBAL_CURSOR_DATE - timedelta(days=LOOKBACK_WINDOW_DAYS) +).isoformat() + "Z" + +PARTITION_SYNC_START_TIME = "2024-01-02T00:00:00Z" + + @pytest.mark.parametrize( "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", [ @@ -354,28 +417,40 @@ def _run_read( [ # Fetch the first page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts?per_page=100&start_time={START_DATE}", { "posts": [ - {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, - {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + {"id": 1, "updated_at": POST_1_UPDATED_AT}, + {"id": 2, "updated_at": POST_2_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + "next_page": f"https://api.example.com/community/posts?per_page=100&start_time={START_DATE}&page=2", }, ), # Fetch the second page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-01T00:00:01Z&page=2", - {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + f"https://api.example.com/community/posts?per_page=100&start_time={START_DATE}&page=2", + {"posts": [{"id": 3, "updated_at": POST_3_UPDATED_AT}]}, ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", { "comments": [ - {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, - {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, - {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + { + "id": 9, + "post_id": 1, + "updated_at": COMMENT_9_OLDEST, # No requests for comment 9, filtered out due to the date + }, + { + "id": 10, + "post_id": 1, + "updated_at": COMMENT_10_UPDATED_AT, + }, + { + "id": 11, + "post_id": 1, + "updated_at": COMMENT_11_UPDATED_AT, + }, ], "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", }, @@ -383,39 +458,59 @@ def _run_read( # Fetch the second page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100&page=2", - {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + { + "comments": [ + { + "id": 12, + "post_id": 1, + "updated_at": COMMENT_12_UPDATED_AT, + } + ] + }, ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", { "votes": [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + { + "id": 100, + "comment_id": 10, + "created_at": VOTE_100_CREATED_AT, + } ], - "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + "next_page": f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", }, ), # Fetch the second page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", { "votes": [ - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} + { + "id": 101, + "comment_id": 10, + "created_at": VOTE_101_CREATED_AT, + } ] }, ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", { "votes": [ - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} + { + "id": 111, + "comment_id": 11, + "created_at": VOTE_111_CREATED_AT, + } ] }, ), # Fetch the first page of votes for comment 12 of post 1 ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={LOOKBACK_DATE}", {"votes": []}, ), # Fetch the first page of comments for post 2 @@ -423,7 +518,11 @@ def _run_read( "https://api.example.com/community/posts/2/comments?per_page=100", { "comments": [ - {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} + { + "id": 20, + "post_id": 2, + "updated_at": COMMENT_20_UPDATED_AT, + } ], "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", }, @@ -431,128 +530,191 @@ def _run_read( # Fetch the second page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100&page=2", - {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + { + "comments": [ + { + "id": 21, + "post_id": 2, + "updated_at": COMMENT_21_UPDATED_AT, + } + ] + }, ), # Fetch the first page of votes for comment 20 of post 2 ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={LOOKBACK_DATE}", { "votes": [ - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} + { + "id": 200, + "comment_id": 20, + "created_at": VOTE_200_CREATED_AT, + } ] }, ), # Fetch the first page of votes for comment 21 of post 2 ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={LOOKBACK_DATE}", { "votes": [ - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} + { + "id": 201, + "comment_id": 21, + "created_at": VOTE_201_CREATED_AT, + } ] }, ), # Fetch the first page of comments for post 3 ( "https://api.example.com/community/posts/3/comments?per_page=100", - {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + { + "comments": [ + { + "id": 30, + "post_id": 3, + "updated_at": COMMENT_30_UPDATED_AT, + } + ] + }, ), # Fetch the first page of votes for comment 30 of post 3 ( "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", { "votes": [ - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} + { + "id": 300, + "comment_id": 30, + "created_at": VOTE_300_CREATED_AT, + } ] }, ), ], # Expected records [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"}, - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_100_CREATED_AT, + "id": 100, + }, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_101_CREATED_AT, + "id": 101, + }, + { + "comment_id": 11, + "comment_updated_at": COMMENT_11_UPDATED_AT, + "created_at": VOTE_111_CREATED_AT, + "id": 111, + }, + { + "comment_id": 20, + "comment_updated_at": COMMENT_20_UPDATED_AT, + "created_at": VOTE_200_CREATED_AT, + "id": 200, + }, + { + "comment_id": 21, + "comment_updated_at": COMMENT_21_UPDATED_AT, + "created_at": VOTE_201_CREATED_AT, + "id": 201, + }, + { + "comment_id": 30, + "comment_updated_at": COMMENT_30_UPDATED_AT, + "created_at": VOTE_300_CREATED_AT, + "id": 300, + }, ], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), - stream_state=AirbyteStateBlob( + { + # This should not happen since parent state is disabled, but I've added this to validate that and + # incoming parent_state is ignored when the parent stream's incremental_dependency is disabled + "parent_state": { + "post_comments": { + "states": [ { - # This should not happen since parent state is disabled, but I've added this to validate that and - # incoming parent_state is ignored when the parent stream's incremental_dependency is disabled - "parent_state": { - "post_comments": { - "states": [ - { - "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, - } - ], - "parent_state": { - "posts": {"updated_at": "2024-01-05T00:00:00Z"} - }, - } - }, - "states": [ - { - "partition": { - "id": 10, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, - }, - { - "partition": { - "id": 11, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, - }, - ], - "lookback_window": 86400, + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, } - ), - ), - ) - ], + ], + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + }, + ], + "state": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + "lookback_window": 86400, + }, # Expected state { "states": [ { - "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": VOTE_100_CREATED_AT}, }, { - "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": VOTE_111_CREATED_AT}, }, { - "cursor": {"created_at": "2024-01-01T00:00:01Z"}, - "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + "partition": { + "id": 12, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": LOOKBACK_DATE}, }, { - "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:00Z"}, + "partition": { + "id": 20, + "parent_slice": {"id": 2, "parent_slice": {}}, + }, + "cursor": {"created_at": VOTE_200_CREATED_AT}, }, { - "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + "partition": { + "id": 21, + "parent_slice": {"id": 2, "parent_slice": {}}, + }, + "cursor": {"created_at": VOTE_201_CREATED_AT}, }, { - "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + "partition": { + "id": 30, + "parent_slice": {"id": 3, "parent_slice": {}}, + }, + "cursor": {"created_at": VOTE_300_CREATED_AT}, }, ], "lookback_window": 1, "parent_state": {}, - "state": {"created_at": "2024-01-15T00:00:00Z"}, + "state": {"created_at": VOTE_100_CREATED_AT}, }, ), ], @@ -564,7 +726,7 @@ def test_incremental_parent_state_no_incremental_dependency( This is a pretty complicated test that syncs a low-code connector stream with three levels of substreams - posts: (ids: 1, 2, 3) - post comments: (parent post 1 with ids: 9, 10, 11, 12; parent post 2 with ids: 20, 21; parent post 3 with id: 30) - - post comment votes: (parent comment 10 with ids: 100, 101; parent comment 11 with id: 102; + - post comment votes: (parent comment 10 with ids: 100, 101; parent comment 11 with id: 111; parent comment 20 with id: 200; parent comment 21 with id: 201, parent comment 30 with id: 300) By setting incremental_dependency to false, parent streams will not use the incoming state and will not update state. @@ -604,24 +766,29 @@ def run_incremental_parent_state_test( initial_state (list): The initial state to start the read operation. expected_states (list): A list of expected final states after the read operation. """ - _stream_name = "post_comment_votes" - config = { - "start_date": "2024-01-01T00:00:01Z", - "credentials": {"email": "email", "api_token": "api_token"}, - } expected_states = [AirbyteStateBlob(s) for s in expected_states] + initial_state = [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(name=STREAM_NAME, namespace=None), + stream_state=AirbyteStateBlob(initial_state), + ), + ) + ] + with requests_mock.Mocker() as m: for url, response in mock_requests: m.get(url, json=response) # Run the initial read - output = _run_read(manifest, config, _stream_name, initial_state) + output = _run_read(manifest, CONFIG, STREAM_NAME, initial_state) # Assert that output_data equals expected_records - assert sorted( - [r.record.data for r in output.records], key=lambda x: orjson.dumps(x) - ) == sorted(expected_records, key=lambda x: orjson.dumps(x)) + assert sorted([r.record.data for r in output.records], key=lambda x: x["id"]) == sorted( + expected_records, key=lambda x: x["id"] + ) # Collect the intermediate states and records produced before each state cumulative_records = [] @@ -643,7 +810,7 @@ def run_incremental_parent_state_test( # For each intermediate state, perform another read starting from that state for state, records_before_state in intermediate_states[:-1]: - output_intermediate = _run_read(manifest, config, _stream_name, [state]) + output_intermediate = _run_read(manifest, CONFIG, STREAM_NAME, [state]) records_from_state = [r.record.data for r in output_intermediate.records] # Combine records produced before the state with records from the new read @@ -686,28 +853,43 @@ def run_incremental_parent_state_test( [ # Fetch the first page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + f"https://api.example.com/community/posts?per_page=100&start_time={PARTITION_SYNC_START_TIME}", { "posts": [ - {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, - {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + {"id": 1, "updated_at": POST_1_UPDATED_AT}, + {"id": 2, "updated_at": POST_2_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + "next_page": ( + f"https://api.example.com/community/posts" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}&page=2" + ), }, ), # Fetch the second page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", - {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + f"https://api.example.com/community/posts?per_page=100&start_time={PARTITION_SYNC_START_TIME}&page=2", + {"posts": [{"id": 3, "updated_at": POST_3_UPDATED_AT}]}, ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", { "comments": [ - {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, - {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, - {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + { + "id": 9, + "post_id": 1, + "updated_at": COMMENT_9_OLDEST, + }, + { + "id": 10, + "post_id": 1, + "updated_at": COMMENT_10_UPDATED_AT, + }, + { + "id": 11, + "post_id": 1, + "updated_at": COMMENT_11_UPDATED_AT, + }, ], "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", }, @@ -715,208 +897,137 @@ def run_incremental_parent_state_test( # Fetch the second page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100&page=2", - {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + {"comments": [{"id": 12, "post_id": 1, "updated_at": COMMENT_12_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", { "votes": [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} + { + "id": 100, + "comment_id": 10, + "created_at": VOTE_100_CREATED_AT, + } ], - "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + "next_page": ( + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={PARTITION_SYNC_START_TIME}" + ), }, ), # Fetch the second page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", - { - "votes": [ - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 101, "comment_id": 10, "created_at": VOTE_101_CREATED_AT}]}, ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", - { - "votes": [ - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 111, "comment_id": 11, "created_at": VOTE_111_CREATED_AT}]}, ), # Fetch the first page of votes for comment 12 of post 1 ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", {"votes": []}, ), # Fetch the first page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100", { - "comments": [ - {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} - ], + "comments": [{"id": 20, "post_id": 2, "updated_at": COMMENT_20_UPDATED_AT}], "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", }, ), # Fetch the second page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100&page=2", - {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + {"comments": [{"id": 21, "post_id": 2, "updated_at": COMMENT_21_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 20 of post 2 ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", - { - "votes": [ - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 200, "comment_id": 20, "created_at": VOTE_200_CREATED_AT}]}, ), # Fetch the first page of votes for comment 21 of post 2 ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", - { - "votes": [ - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} - ] - }, + f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 201, "comment_id": 21, "created_at": VOTE_201_CREATED_AT}]}, ), # Fetch the first page of comments for post 3 ( "https://api.example.com/community/posts/3/comments?per_page=100", - {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + {"comments": [{"id": 30, "post_id": 3, "updated_at": COMMENT_30_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 30 of post 3 ( - "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", - { - "votes": [ - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} - ] - }, - ), - # Requests with intermediate states - # Fetch votes for comment 10 of post 1 - ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-15T00:00:00Z", - { - "votes": [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} - ], - }, - ), - # Fetch votes for comment 11 of post 1 - ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-13T00:00:00Z", - { - "votes": [ - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} - ], - }, - ), - # Fetch votes for comment 12 of post 1 - ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-15T00:00:00Z", - { - "votes": [], - }, - ), - # Fetch votes for comment 20 of post 2 - ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-12T00:00:00Z", - { - "votes": [ - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} - ] - }, - ), - # Fetch votes for comment 21 of post 2 - ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-12T00:00:15Z", - { - "votes": [ - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} - ] - }, + f"https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 300, "comment_id": 30, "created_at": VOTE_300_CREATED_AT}]}, ), ], # Expected records [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"}, - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_100_CREATED_AT, + "id": 100, + }, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_101_CREATED_AT, + "id": 101, + }, + { + "comment_id": 11, + "comment_updated_at": COMMENT_11_UPDATED_AT, + "created_at": VOTE_111_CREATED_AT, + "id": 111, + }, + { + "comment_id": 20, + "comment_updated_at": COMMENT_20_UPDATED_AT, + "created_at": VOTE_200_CREATED_AT, + "id": 200, + }, + { + "comment_id": 21, + "comment_updated_at": COMMENT_21_UPDATED_AT, + "created_at": VOTE_201_CREATED_AT, + "id": 201, + }, + { + "comment_id": 30, + "comment_updated_at": COMMENT_30_UPDATED_AT, + "created_at": VOTE_300_CREATED_AT, + "id": 300, + }, ], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), - stream_state=AirbyteStateBlob( - { - "parent_state": { - "post_comments": { - "states": [ - { - "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, - } - ], - "parent_state": { - "posts": {"updated_at": "2024-01-05T00:00:00Z"} - }, - } - }, - "states": [ - { - "partition": { - "id": 10, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, - }, - { - "partition": { - "id": 11, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, - }, - ], - "lookback_window": 86400, - } - ), - ), - ) - ], + {"created_at": PARTITION_SYNC_START_TIME}, # Expected state { - "state": {"created_at": "2024-01-15T00:00:00Z"}, + "state": {"created_at": VOTE_100_CREATED_AT}, "parent_state": { "post_comments": { "use_global_cursor": False, - "state": {"updated_at": "2024-01-25T00:00:00Z"}, - "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "state": {"updated_at": COMMENT_10_UPDATED_AT}, # 10 is the "latest" + "parent_state": { + "posts": {"updated_at": POST_1_UPDATED_AT} + }, # post 1 is the latest "lookback_window": 1, "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_10_UPDATED_AT}, }, { "partition": {"id": 2, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_20_UPDATED_AT}, }, { "partition": {"id": 3, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_30_UPDATED_AT}, }, ], } @@ -925,27 +1036,27 @@ def run_incremental_parent_state_test( "states": [ { "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + "cursor": {"created_at": VOTE_100_CREATED_AT}, }, { "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + "cursor": {"created_at": VOTE_111_CREATED_AT}, }, { "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + "cursor": {"created_at": PARTITION_SYNC_START_TIME}, }, { "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:00Z"}, + "cursor": {"created_at": VOTE_200_CREATED_AT}, }, { "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + "cursor": {"created_at": VOTE_201_CREATED_AT}, }, { "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + "cursor": {"created_at": VOTE_300_CREATED_AT}, }, ], }, @@ -955,19 +1066,12 @@ def run_incremental_parent_state_test( def test_incremental_parent_state( test_name, manifest, mock_requests, expected_records, initial_state, expected_state ): - additional_expected_state = copy.deepcopy(expected_state) - # State for empty partition (comment 12), when the global cursor is used for intermediate states - empty_state = { - "cursor": {"created_at": "2024-01-01T00:00:01Z"}, - "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, - } - additional_expected_state["states"].append(empty_state) run_incremental_parent_state_test( manifest, mock_requests, expected_records, initial_state, - [expected_state, additional_expected_state], + [expected_state], ) @@ -980,160 +1084,176 @@ def test_incremental_parent_state( [ # Fetch the first page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts?per_page=100&start_time={PARTITION_SYNC_START_TIME}", { "posts": [ - {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, - {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + {"id": 1, "updated_at": POST_1_UPDATED_AT}, + {"id": 2, "updated_at": POST_2_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-02T00:00:00Z&page=2", + "next_page": ( + f"https://api.example.com/community/posts?per_page=100" + f"&start_time={PARTITION_SYNC_START_TIME}&page=2" + ), }, ), # Fetch the second page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-02T00:00:00Z&page=2", - {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + f"https://api.example.com/community/posts?per_page=100" + f"&start_time={PARTITION_SYNC_START_TIME}&page=2", + {"posts": [{"id": 3, "updated_at": POST_3_UPDATED_AT}]}, ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", { "comments": [ - {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, - {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, - {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + {"id": 9, "post_id": 1, "updated_at": COMMENT_9_OLDEST}, + {"id": 10, "post_id": 1, "updated_at": COMMENT_10_UPDATED_AT}, + {"id": 11, "post_id": 1, "updated_at": COMMENT_11_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "next_page": ( + "https://api.example.com/community/posts/1/comments" + "?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100&page=2", - {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + {"comments": [{"id": 12, "post_id": 1, "updated_at": COMMENT_12_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", { - "votes": [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} - ], - "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-02T00:00:00Z", + "votes": [{"id": 100, "comment_id": 10, "created_at": VOTE_100_CREATED_AT}], + "next_page": ( + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={PARTITION_SYNC_START_TIME}" + ), }, ), # Fetch the second page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-02T00:00:00Z", - { - "votes": [ - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 101, "comment_id": 10, "created_at": VOTE_101_CREATED_AT}]}, ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-02T00:00:00Z", - { - "votes": [ - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/1/comments/11/votes" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 111, "comment_id": 11, "created_at": VOTE_111_CREATED_AT}]}, ), # Fetch the first page of votes for comment 12 of post 1 ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/12/votes" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", {"votes": []}, ), # Fetch the first page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100", { - "comments": [ - {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} - ], - "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "comments": [{"id": 20, "post_id": 2, "updated_at": COMMENT_20_UPDATED_AT}], + "next_page": ( + "https://api.example.com/community/posts/2/comments" + "?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100&page=2", - {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + {"comments": [{"id": 21, "post_id": 2, "updated_at": COMMENT_21_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 20 of post 2 ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-02T00:00:00Z", - { - "votes": [ - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/2/comments/20/votes" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 200, "comment_id": 20, "created_at": VOTE_200_CREATED_AT}]}, ), # Fetch the first page of votes for comment 21 of post 2 ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-02T00:00:00Z", - { - "votes": [ - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} - ] - }, + f"https://api.example.com/community/posts/2/comments/21/votes" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 201, "comment_id": 21, "created_at": VOTE_201_CREATED_AT}]}, ), # Fetch the first page of comments for post 3 ( "https://api.example.com/community/posts/3/comments?per_page=100", - {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + {"comments": [{"id": 30, "post_id": 3, "updated_at": COMMENT_30_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 30 of post 3 ( - "https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time=2024-01-02T00:00:00Z", - { - "votes": [ - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/3/comments/30/votes" + f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + {"votes": [{"id": 300, "comment_id": 30, "created_at": VOTE_300_CREATED_AT}]}, ), ], # Expected records [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, - {"id": 200, "comment_id": 20, "created_at": "2024-01-12T00:00:00Z"}, - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_100_CREATED_AT, + "id": 100, + }, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_101_CREATED_AT, + "id": 101, + }, + { + "comment_id": 11, + "comment_updated_at": COMMENT_11_UPDATED_AT, + "created_at": VOTE_111_CREATED_AT, + "id": 111, + }, + { + "comment_id": 20, + "comment_updated_at": COMMENT_20_UPDATED_AT, + "created_at": VOTE_200_CREATED_AT, + "id": 200, + }, + { + "comment_id": 21, + "comment_updated_at": COMMENT_21_UPDATED_AT, + "created_at": VOTE_201_CREATED_AT, + "id": 201, + }, + { + "comment_id": 30, + "comment_updated_at": COMMENT_30_UPDATED_AT, + "created_at": VOTE_300_CREATED_AT, + "id": 300, + }, ], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), - stream_state=AirbyteStateBlob({"created_at": "2024-01-02T00:00:00Z"}), - ), - ) - ], + {"created_at": PARTITION_SYNC_START_TIME}, # Expected state { - "state": {"created_at": "2024-01-15T00:00:00Z"}, + "state": {"created_at": VOTE_100_CREATED_AT}, "parent_state": { "post_comments": { "use_global_cursor": False, - "state": {"updated_at": "2024-01-25T00:00:00Z"}, - "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "state": {"updated_at": COMMENT_10_UPDATED_AT}, + "parent_state": {"posts": {"updated_at": POST_1_UPDATED_AT}}, "lookback_window": 1, "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_10_UPDATED_AT}, }, { "partition": {"id": 2, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_20_UPDATED_AT}, }, { "partition": {"id": 3, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_30_UPDATED_AT}, }, ], } @@ -1142,27 +1262,27 @@ def test_incremental_parent_state( "states": [ { "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + "cursor": {"created_at": VOTE_100_CREATED_AT}, }, { "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + "cursor": {"created_at": VOTE_111_CREATED_AT}, }, { "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + "cursor": {"created_at": PARTITION_SYNC_START_TIME}, }, { "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:00Z"}, + "cursor": {"created_at": VOTE_200_CREATED_AT}, }, { "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + "cursor": {"created_at": VOTE_201_CREATED_AT}, }, { "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + "cursor": {"created_at": VOTE_300_CREATED_AT}, }, ], }, @@ -1195,15 +1315,19 @@ def test_incremental_parent_state_migration( [ # Fetch the first page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}", { "posts": [], - "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + "next_page": ( + f"https://api.example.com/community/posts?per_page=100" + f"&start_time={PARENT_POSTS_CURSOR}&page=2" + ), }, ), # Fetch the second page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + f"https://api.example.com/community/posts?per_page=100" + f"&start_time={PARENT_POSTS_CURSOR}&page=2", {"posts": []}, ), # Fetch the first page of comments for post 1 @@ -1211,7 +1335,9 @@ def test_incremental_parent_state_migration( "https://api.example.com/community/posts/1/comments?per_page=100", { "comments": [], - "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "next_page": ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 1 @@ -1221,25 +1347,28 @@ def test_incremental_parent_state_migration( ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", { "votes": [], - "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + "next_page": ( + "https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={START_DATE}" + ), }, ), # Fetch the second page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={START_DATE}", {"votes": []}, ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": []}, ), # Fetch the first page of votes for comment 12 of post 1 ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={START_DATE}", {"votes": []}, ), # Fetch the first page of comments for post 2 @@ -1247,7 +1376,9 @@ def test_incremental_parent_state_migration( "https://api.example.com/community/posts/2/comments?per_page=100", { "comments": [], - "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "next_page": ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 2 @@ -1257,12 +1388,12 @@ def test_incremental_parent_state_migration( ), # Fetch the first page of votes for comment 20 of post 2 ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={START_DATE}", {"votes": []}, ), # Fetch the first page of votes for comment 21 of post 2 ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={START_DATE}", {"votes": []}, ), # Fetch the first page of comments for post 3 @@ -1276,81 +1407,73 @@ def test_incremental_parent_state_migration( {"votes": []}, ), ], - # Expected records + # Expected records (empty) [], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), - stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ { - "parent_state": { - "post_comments": { - "states": [ - { - "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, - } - ], - "parent_state": { - "posts": {"updated_at": "2024-01-05T00:00:00Z"} - }, - } - }, - "states": [ - { - "partition": { - "id": 10, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, - }, - { - "partition": { - "id": 11, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, - }, - ], - "state": {"created_at": "2024-01-03T00:00:00Z"}, - "lookback_window": 1, + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, } - ), - ), - ) - ], + ], + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + }, + ], + "state": {"created_at": INITIAL_GLOBAL_CURSOR}, + "lookback_window": 1, + }, # Expected state { - "lookback_window": 1, - "state": {"created_at": "2024-01-03T00:00:00Z"}, "parent_state": { "post_comments": { - "use_global_cursor": False, - "state": {}, - "parent_state": {"posts": {"updated_at": "2024-01-05T00:00:00Z"}}, "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, } ], + "state": {}, + "use_global_cursor": False, + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, } }, "states": [ { - "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, }, { - "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, }, ], + "state": {"created_at": INITIAL_GLOBAL_CURSOR}, + "lookback_window": 1, }, ), ], @@ -1381,89 +1504,104 @@ def test_incremental_parent_state_no_slices( [ # Fetch the first page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}", { "posts": [ - {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, - {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + {"id": 1, "updated_at": POST_1_UPDATED_AT}, + {"id": 2, "updated_at": POST_2_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + "next_page": ( + f"https://api.example.com/community/posts?per_page=100" + f"&start_time={PARENT_POSTS_CURSOR}&page=2" + ), }, ), # Fetch the second page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", - {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + f"https://api.example.com/community/posts?per_page=100" + f"&start_time={PARENT_POSTS_CURSOR}&page=2", + {"posts": [{"id": 3, "updated_at": POST_3_UPDATED_AT}]}, ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", { "comments": [ - {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, - {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, - {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + {"id": 9, "post_id": 1, "updated_at": COMMENT_9_OLDEST}, + {"id": 10, "post_id": 1, "updated_at": COMMENT_10_UPDATED_AT}, + {"id": 11, "post_id": 1, "updated_at": COMMENT_11_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "next_page": ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100&page=2", - {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + {"comments": [{"id": 12, "post_id": 1, "updated_at": COMMENT_12_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", { "votes": [], - "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + "next_page": ( + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}" + ), }, ), # Fetch the second page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", {"votes": []}, ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/11/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": []}, ), # Fetch the first page of votes for comment 12 of post 1 ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/12/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": []}, ), # Fetch the first page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100", { - "comments": [ - {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} - ], - "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "comments": [{"id": 20, "post_id": 2, "updated_at": COMMENT_20_UPDATED_AT}], + "next_page": ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100&page=2", - {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + {"comments": [{"id": 21, "post_id": 2, "updated_at": COMMENT_21_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 20 of post 2 ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + f"https://api.example.com/community/posts/2/comments/20/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": []}, ), # Fetch the first page of votes for comment 21 of post 2 ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-03T00:00:00Z", + f"https://api.example.com/community/posts/2/comments/21/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": []}, ), # Fetch the first page of comments for post 3 ( "https://api.example.com/community/posts/3/comments?per_page=100", - {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + {"comments": [{"id": 30, "post_id": 3, "updated_at": COMMENT_30_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 30 of post 3 ( @@ -1474,106 +1612,92 @@ def test_incremental_parent_state_no_slices( # Expected records [], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), - stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ { - "parent_state": { - "post_comments": { - "states": [ - { - "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, - } - ], - "parent_state": { - "posts": {"updated_at": "2024-01-05T00:00:00Z"} - }, - } - }, - "states": [ - { - "partition": { - "id": 10, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, - }, - { - "partition": { - "id": 11, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, - }, - ], - "use_global_cursor": True, - "state": {"created_at": "2024-01-03T00:00:00Z"}, - "lookback_window": 0, + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, } - ), - ), - ) - ], + ], + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, + } + }, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + }, + ], + "use_global_cursor": True, + "state": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + "lookback_window": 0, + }, # Expected state { "lookback_window": 1, - "state": {"created_at": "2024-01-03T00:00:00Z"}, + "state": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, "states": [ { "partition": { "id": 10, "parent_slice": {"id": 1, "parent_slice": {}}, }, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, }, { "partition": { "id": 11, "parent_slice": {"id": 1, "parent_slice": {}}, }, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, }, { - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, }, { - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, }, { - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, }, { - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, }, ], "parent_state": { "post_comments": { "use_global_cursor": False, - "state": {"updated_at": "2024-01-25T00:00:00Z"}, - "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, + "state": {"updated_at": COMMENT_10_UPDATED_AT}, + "parent_state": {"posts": {"updated_at": POST_1_UPDATED_AT}}, "lookback_window": 1, "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_10_UPDATED_AT}, }, { "partition": {"id": 2, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_20_UPDATED_AT}, }, { "partition": {"id": 3, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, + "cursor": {"updated_at": COMMENT_30_UPDATED_AT}, }, ], } @@ -1608,226 +1732,220 @@ def test_incremental_parent_state_no_records( [ # Fetch the first page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z", + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}", { "posts": [ - {"id": 1, "updated_at": "2024-01-30T00:00:00Z"}, - {"id": 2, "updated_at": "2024-01-29T00:00:00Z"}, + {"id": 1, "updated_at": POST_1_UPDATED_AT}, + {"id": 2, "updated_at": POST_2_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", + "next_page": ( + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}&page=2" + ), }, ), # Fetch the second page of posts ( - "https://api.example.com/community/posts?per_page=100&start_time=2024-01-05T00:00:00Z&page=2", - {"posts": [{"id": 3, "updated_at": "2024-01-28T00:00:00Z"}]}, + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}&page=2", + {"posts": [{"id": 3, "updated_at": POST_3_UPDATED_AT}]}, ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", { "comments": [ - {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, - {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, - {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, + {"id": 9, "post_id": 1, "updated_at": COMMENT_9_OLDEST}, + {"id": 10, "post_id": 1, "updated_at": COMMENT_10_UPDATED_AT}, + {"id": 11, "post_id": 1, "updated_at": COMMENT_11_UPDATED_AT}, ], - "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "next_page": ( + "https://api.example.com/community/posts/1/comments?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100&page=2", - {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, + {"comments": [{"id": 12, "post_id": 1, "updated_at": COMMENT_12_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time=2024-01-02T00:00:00Z", + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", { - "votes": [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"} - ], - "next_page": "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", + "votes": [{"id": 100, "comment_id": 10, "created_at": VOTE_100_CREATED_AT}], + "next_page": ( + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}" + ), }, ), # Fetch the second page of votes for comment 10 of post 1 ( - "https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time=2024-01-01T00:00:01Z", - { - "votes": [ - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", + {"votes": [{"id": 101, "comment_id": 10, "created_at": VOTE_101_CREATED_AT}]}, ), # Fetch the first page of votes for comment 11 of post 1 ( - "https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time=2024-01-03T00:00:00Z", - { - "votes": [ - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"} - ] - }, + f"https://api.example.com/community/posts/1/comments/11/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", + {"votes": [{"id": 111, "comment_id": 11, "created_at": VOTE_111_CREATED_AT}]}, ), # Fetch the first page of votes for comment 12 of post 1 ( - "https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time=2024-01-01T00:00:01Z", + f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={LOOKBACK_DATE}", {"votes": []}, ), # Fetch the first page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100", { - "comments": [ - {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} - ], - "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "comments": [{"id": 20, "post_id": 2, "updated_at": COMMENT_20_UPDATED_AT}], + "next_page": ( + "https://api.example.com/community/posts/2/comments?per_page=100&page=2" + ), }, ), # Fetch the second page of comments for post 2 ( "https://api.example.com/community/posts/2/comments?per_page=100&page=2", - {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, + {"comments": [{"id": 21, "post_id": 2, "updated_at": COMMENT_21_UPDATED_AT}]}, ), - # Fetch the first page of votes for comment 20 of post 2 + # Fetch the first page of votes for comment 20 of post 2 - 404 error ( - "https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time=2024-01-01T00:00:01Z", - { - "status_code": 500, - "json": {"error": "Internal Server Error"}, - }, + f"https://api.example.com/community/posts/2/comments/20/votes" + f"?per_page=100&start_time={LOOKBACK_DATE}", + None, ), # Fetch the first page of votes for comment 21 of post 2 ( - "https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time=2024-01-01T00:00:01Z", - { - "votes": [ - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"} - ] - }, + f"https://api.example.com/community/posts/2/comments/21/votes" + f"?per_page=100&start_time={LOOKBACK_DATE}", + {"votes": [{"id": 201, "comment_id": 21, "created_at": VOTE_201_CREATED_AT}]}, ), # Fetch the first page of comments for post 3 ( "https://api.example.com/community/posts/3/comments?per_page=100", - {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, + {"comments": [{"id": 30, "post_id": 3, "updated_at": COMMENT_30_UPDATED_AT}]}, ), # Fetch the first page of votes for comment 30 of post 3 ( "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", - { - "votes": [ - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"} - ] - }, + {"votes": [{"id": 300, "comment_id": 30, "created_at": VOTE_300_CREATED_AT}]}, ), ], # Expected records [ - {"id": 100, "comment_id": 10, "created_at": "2024-01-15T00:00:00Z"}, - {"id": 101, "comment_id": 10, "created_at": "2024-01-14T00:00:00Z"}, - {"id": 102, "comment_id": 11, "created_at": "2024-01-13T00:00:00Z"}, - {"id": 201, "comment_id": 21, "created_at": "2024-01-12T00:00:15Z"}, - {"id": 300, "comment_id": 30, "created_at": "2024-01-10T00:00:00Z"}, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_100_CREATED_AT, + "id": 100, + }, + { + "comment_id": 10, + "comment_updated_at": COMMENT_10_UPDATED_AT, + "created_at": VOTE_101_CREATED_AT, + "id": 101, + }, + { + "comment_id": 11, + "comment_updated_at": COMMENT_11_UPDATED_AT, + "created_at": VOTE_111_CREATED_AT, + "id": 111, + }, + { + "comment_id": 21, + "comment_updated_at": COMMENT_21_UPDATED_AT, + "created_at": VOTE_201_CREATED_AT, + "id": 201, + }, + { + "comment_id": 30, + "comment_updated_at": COMMENT_30_UPDATED_AT, + "created_at": VOTE_300_CREATED_AT, + "id": 300, + }, ], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor( - name="post_comment_votes", namespace=None - ), - stream_state=AirbyteStateBlob( + { + "parent_state": { + "post_comments": { + "states": [ { - "parent_state": { - "post_comments": { - "states": [ - { - "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2023-01-04T00:00:00Z"}, - } - ], - "parent_state": { - "posts": {"updated_at": "2024-01-05T00:00:00Z"} - }, - } - }, - "states": [ - { - "partition": { - "id": 10, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-02T00:00:00Z"}, - }, - { - "partition": { - "id": 11, - "parent_slice": {"id": 1, "parent_slice": {}}, - }, - "cursor": {"created_at": "2024-01-03T00:00:00Z"}, - }, - ], + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, } - ), - ), - ) - ], + ], + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, + } + }, + "state": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + "lookback_window": 86400, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + }, + ], + }, # Expected state { - "state": {"created_at": "2024-01-15T00:00:00Z"}, + # The global state, lookback window and the parent state are the same because sync failed for comment 20 "parent_state": { "post_comments": { - "use_global_cursor": False, - "state": {"updated_at": "2024-01-25T00:00:00Z"}, - "parent_state": {"posts": {"updated_at": "2024-01-30T00:00:00Z"}}, - "lookback_window": 1, "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-25T00:00:00Z"}, - }, - { - "partition": {"id": 2, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, - }, - { - "partition": {"id": 3, "parent_slice": {}}, - "cursor": {"updated_at": "2024-01-09T00:00:00Z"}, - }, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, + } ], + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, } }, - "lookback_window": 1, + "state": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + "lookback_window": 86400, "states": [ { "partition": {"id": 10, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-15T00:00:00Z"}, + "cursor": {"created_at": VOTE_100_CREATED_AT}, }, { "partition": {"id": 11, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-13T00:00:00Z"}, + "cursor": {"created_at": VOTE_111_CREATED_AT}, }, { "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + "cursor": {"created_at": LOOKBACK_DATE}, }, { "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-01T00:00:01Z"}, + "cursor": {"created_at": LOOKBACK_DATE}, }, { "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-12T00:00:15Z"}, + "cursor": {"created_at": VOTE_201_CREATED_AT}, }, { "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, - "cursor": {"created_at": "2024-01-10T00:00:00Z"}, + "cursor": {"created_at": VOTE_300_CREATED_AT}, }, ], }, ), ], ) -def test_incremental_error__parent_state( +def test_incremental_substream_error( test_name, manifest, mock_requests, expected_records, initial_state, expected_state ): run_mocked_test( @@ -2048,30 +2166,20 @@ def test_incremental_error__parent_state( {"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}, ], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor(name="post_comments", namespace=None), - stream_state=AirbyteStateBlob( - { - "state": {"updated_at": "2024-01-08T00:00:00Z"}, - "states": [ - { - "cursor": {"updated_at": "2024-01-24T00:00:00Z"}, - "partition": {"id": "1"}, - }, - { - "cursor": {"updated_at": "2024-01-21T05:00:00Z"}, - "partition": {"id": "2"}, - }, - ], - "use_global_cursor": False, - } - ), - ), - ) - ], + { + "state": {"updated_at": "2024-01-08T00:00:00Z"}, + "states": [ + { + "cursor": {"updated_at": "2024-01-24T00:00:00Z"}, + "partition": {"id": "1"}, + }, + { + "cursor": {"updated_at": "2024-01-21T05:00:00Z"}, + "partition": {"id": "2"}, + }, + ], + "use_global_cursor": False, + }, # Expected state { "lookback_window": 1, @@ -2155,30 +2263,20 @@ def test_incremental_list_partition_router( {"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}, ], # Initial state - [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor(name="post_comments", namespace=None), - stream_state=AirbyteStateBlob( - { - "state": {"updated_at": "2024-01-08T00:00:00Z"}, - "states": [ - { - "cursor": {"updated_at": "2024-01-20T00:00:00Z"}, - "partition": {"id": "1"}, - }, - { - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, - "partition": {"id": "2"}, - }, - ], - "use_global_cursor": False, - } - ), - ), - ) - ], + { + "state": {"updated_at": "2024-01-08T00:00:00Z"}, + "states": [ + { + "cursor": {"updated_at": "2024-01-20T00:00:00Z"}, + "partition": {"id": "1"}, + }, + { + "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "partition": {"id": "2"}, + }, + ], + "use_global_cursor": False, + }, # Expected state { "lookback_window": 0, From d4d52b9629398d57615e18ea39f3f71051bf671f Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 22 Jan 2025 17:57:55 +0200 Subject: [PATCH 32/34] Add comments for state format --- .../declarative/incremental/concurrent_partition_cursor.py | 2 ++ .../sources/declarative/parsers/model_to_component_factory.py | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index be7167493..906ad1b5c 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -51,6 +51,8 @@ class ConcurrentPerPartitionCursor(Cursor): - **Global Cursor Fallback** New partitions use global state as the initial state to progress the state for deleted or new partitions. The history data added after the initial sync will be missing. + + CurrentPerPartitionCursor expects the state of the ConcurrentCursor to follow the format {cursor_field: cursor_value}. """ DEFAULT_MAX_PARTITIONS_NUMBER = 10000 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 81f3bb7ff..ba18906a1 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -937,7 +937,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( connector_state_converter = CustomFormatConcurrentStreamStateConverter( datetime_format=datetime_format, input_datetime_formats=datetime_based_cursor_model.cursor_datetime_formats, - is_sequential_state=True, + is_sequential_state=True, # ConcurrentPerPartitionCursor only works with sequential state cursor_granularity=cursor_granularity, ) From 0c7c4def46ef393ce8066a3df62b62bc63e91e88 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 22 Jan 2025 21:34:39 +0200 Subject: [PATCH 33/34] Add emitting state after closing every partition --- .../declarative/incremental/concurrent_partition_cursor.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 906ad1b5c..b71890cce 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -147,6 +147,7 @@ def close_partition(self, partition: Partition) -> None: < cursor.state[self.cursor_field.cursor_field_key] ): self._new_global_cursor = copy.deepcopy(cursor.state) + self._emit_state_message() def ensure_at_least_one_state_emitted(self) -> None: """ From 6277e1027d1e23e559d6a106d167b3417e8ead82 Mon Sep 17 00:00:00 2001 From: Anatolii Yatsuk Date: Wed, 22 Jan 2025 21:35:00 +0200 Subject: [PATCH 34/34] Add reqeust validation to unit tests --- .../test_concurrent_perpartitioncursor.py | 305 ++++++++++-------- 1 file changed, 163 insertions(+), 142 deletions(-) diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index b0abfc6e7..8862600d5 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -1,9 +1,9 @@ # Copyright (c) 2024 Airbyte, Inc., all rights reserved. -import copy from copy import deepcopy from datetime import datetime, timedelta from typing import Any, List, Mapping, MutableMapping, Optional, Union +from urllib.parse import unquote import pytest from orjson import orjson @@ -348,6 +348,15 @@ def run_mocked_test( final_state = output.state_messages[-1].state.stream.stream_state assert final_state.__dict__ == expected_state + # Verify that each request was made exactly once + for url, _ in mock_requests: + request_count = len( + [req for req in m.request_history if unquote(req.url) == unquote(url)] + ) + assert ( + request_count == 1 + ), f"URL {url} was called {request_count} times, expected exactly once." + def _run_read( manifest: Mapping[str, Any], @@ -384,15 +393,15 @@ def _run_read( LOOKBACK_WINDOW_DAYS = 1 # Lookback window duration in days # Votes Date Constants -VOTE_100_CREATED_AT = "2024-01-15T00:00:00Z" # Latest vote in partition 1 -VOTE_101_CREATED_AT = "2024-01-14T00:00:00Z" # Second-latest vote in partition 1 -VOTE_111_CREATED_AT = "2024-01-13T00:00:00Z" # Latest vote in partition 2 -VOTE_200_CREATED_AT = "2024-01-12T00:00:00Z" # Latest vote in partition 3 -VOTE_201_CREATED_AT = "2024-01-12T00:00:15Z" # Second-latest vote in partition 3 -VOTE_300_CREATED_AT = "2024-01-10T00:00:00Z" # Latest vote in partition 4 +VOTE_100_CREATED_AT = "2024-01-15T00:00:00Z" # Latest vote in partition 10 +VOTE_101_CREATED_AT = "2024-01-14T00:00:00Z" # Second-latest vote in partition 10 +VOTE_111_CREATED_AT = "2024-01-13T00:00:00Z" # Latest vote in partition 11 +VOTE_200_CREATED_AT = "2024-01-12T00:00:00Z" # Latest vote in partition 20 +VOTE_210_CREATED_AT = "2024-01-12T00:00:15Z" # Latest vote in partition 21 +VOTE_300_CREATED_AT = "2024-01-10T00:00:00Z" # Latest vote in partition 30 # Initial State Constants -PARENT_COMMENT_CURSOR_PARTITION_1 = "2023-01-04T00:00:00Z" # Parent comment cursor (partition 1) +PARENT_COMMENT_CURSOR_PARTITION_1 = "2023-01-04T00:00:00Z" # Parent comment cursor (partition) PARENT_POSTS_CURSOR = "2024-01-05T00:00:00Z" # Parent posts cursor (expected in state) INITIAL_STATE_PARTITION_10_CURSOR = "2024-01-02T00:00:01Z" @@ -559,9 +568,9 @@ def _run_read( { "votes": [ { - "id": 201, + "id": 210, "comment_id": 21, - "created_at": VOTE_201_CREATED_AT, + "created_at": VOTE_210_CREATED_AT, } ] }, @@ -581,7 +590,7 @@ def _run_read( ), # Fetch the first page of votes for comment 30 of post 3 ( - "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + f"https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time={LOOKBACK_DATE}", { "votes": [ { @@ -622,8 +631,8 @@ def _run_read( { "comment_id": 21, "comment_updated_at": COMMENT_21_UPDATED_AT, - "created_at": VOTE_201_CREATED_AT, - "id": 201, + "created_at": VOTE_210_CREATED_AT, + "id": 210, }, { "comment_id": 30, @@ -702,7 +711,7 @@ def _run_read( "id": 21, "parent_slice": {"id": 2, "parent_slice": {}}, }, - "cursor": {"created_at": VOTE_201_CREATED_AT}, + "cursor": {"created_at": VOTE_210_CREATED_AT}, }, { "partition": { @@ -727,7 +736,7 @@ def test_incremental_parent_state_no_incremental_dependency( - posts: (ids: 1, 2, 3) - post comments: (parent post 1 with ids: 9, 10, 11, 12; parent post 2 with ids: 20, 21; parent post 3 with id: 30) - post comment votes: (parent comment 10 with ids: 100, 101; parent comment 11 with id: 111; - parent comment 20 with id: 200; parent comment 21 with id: 201, parent comment 30 with id: 300) + parent comment 20 with id: 200; parent comment 21 with id: 210, parent comment 30 with id: 300) By setting incremental_dependency to false, parent streams will not use the incoming state and will not update state. The post_comment_votes substream is incremental and will emit state messages We verify this by ensuring that mocked @@ -746,7 +755,12 @@ def test_incremental_parent_state_no_incremental_dependency( def run_incremental_parent_state_test( - manifest, mock_requests, expected_records, initial_state, expected_states + manifest, + mock_requests, + expected_records, + num_intermediate_states, + initial_state, + expected_states, ): """ Run an incremental parent state test for the specified stream. @@ -763,11 +777,10 @@ def run_incremental_parent_state_test( manifest (dict): The manifest configuration for the stream. mock_requests (list): A list of tuples containing URL and response data for mocking API requests. expected_records (list): The expected records to compare against the output. + num_intermediate_states (int): The number of intermediate states to expect. initial_state (list): The initial state to start the read operation. expected_states (list): A list of expected final states after the read operation. """ - expected_states = [AirbyteStateBlob(s) for s in expected_states] - initial_state = [ AirbyteStateMessage( type=AirbyteStateType.STREAM, @@ -796,7 +809,7 @@ def run_incremental_parent_state_test( final_states = [] # To store the final state after each read # Store the final state after the initial read - final_states.append(output.state_messages[-1].state.stream.stream_state) + final_states.append(output.state_messages[-1].state.stream.stream_state.__dict__) for message in output.records_and_state_messages: if message.type.value == "RECORD": @@ -808,6 +821,9 @@ def run_incremental_parent_state_test( records_before_state = cumulative_records.copy() intermediate_states.append((state, records_before_state)) + # Assert that the number of intermediate states is as expected + assert len(intermediate_states) - 1 == num_intermediate_states + # For each intermediate state, perform another read starting from that state for state, records_before_state in intermediate_states[:-1]: output_intermediate = _run_read(manifest, CONFIG, STREAM_NAME, [state]) @@ -826,13 +842,13 @@ def run_incremental_parent_state_test( {orjson.dumps(record): record for record in expected_records}.values() ) assert ( - sorted(cumulative_records_state_deduped, key=lambda x: orjson.dumps(x)) - == sorted(expected_records_set, key=lambda x: orjson.dumps(x)) + sorted(cumulative_records_state_deduped, key=lambda x: x["id"]) + == sorted(expected_records_set, key=lambda x: x["id"]) ), f"Records mismatch with intermediate state {state}. Expected {expected_records}, got {cumulative_records_state_deduped}" # Store the final state after each intermediate read final_state_intermediate = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) + message.state.stream.stream_state.__dict__ for message in output_intermediate.state_messages ] final_states.append(final_state_intermediate[-1]) @@ -845,7 +861,7 @@ def run_incremental_parent_state_test( @pytest.mark.parametrize( - "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", + "test_name, manifest, mock_requests, expected_records, num_intermediate_states, initial_state, expected_state", [ ( "test_incremental_parent_state", @@ -853,7 +869,7 @@ def run_incremental_parent_state_test( [ # Fetch the first page of posts ( - f"https://api.example.com/community/posts?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}", { "posts": [ {"id": 1, "updated_at": POST_1_UPDATED_AT}, @@ -861,13 +877,13 @@ def run_incremental_parent_state_test( ], "next_page": ( f"https://api.example.com/community/posts" - f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}&page=2" + f"?per_page=100&start_time={PARENT_POSTS_CURSOR}&page=2" ), }, ), # Fetch the second page of posts ( - f"https://api.example.com/community/posts?per_page=100&start_time={PARTITION_SYNC_START_TIME}&page=2", + f"https://api.example.com/community/posts?per_page=100&start_time={PARENT_POSTS_CURSOR}&page=2", {"posts": [{"id": 3, "updated_at": POST_3_UPDATED_AT}]}, ), # Fetch the first page of comments for post 1 @@ -901,7 +917,7 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 10 of post 1 ( - f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", { "votes": [ { @@ -911,23 +927,26 @@ def run_incremental_parent_state_test( } ], "next_page": ( - f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={PARTITION_SYNC_START_TIME}" + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}" ), }, ), # Fetch the second page of votes for comment 10 of post 1 ( - f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts/1/comments/10/votes" + f"?per_page=100&page=2&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", {"votes": [{"id": 101, "comment_id": 10, "created_at": VOTE_101_CREATED_AT}]}, ), # Fetch the first page of votes for comment 11 of post 1 ( - f"https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts/1/comments/11/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": [{"id": 111, "comment_id": 11, "created_at": VOTE_111_CREATED_AT}]}, ), # Fetch the first page of votes for comment 12 of post 1 ( - f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={LOOKBACK_DATE}", {"votes": []}, ), # Fetch the first page of comments for post 2 @@ -945,13 +964,13 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 20 of post 2 ( - f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={LOOKBACK_DATE}", {"votes": [{"id": 200, "comment_id": 20, "created_at": VOTE_200_CREATED_AT}]}, ), # Fetch the first page of votes for comment 21 of post 2 ( - f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", - {"votes": [{"id": 201, "comment_id": 21, "created_at": VOTE_201_CREATED_AT}]}, + f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={LOOKBACK_DATE}", + {"votes": [{"id": 210, "comment_id": 21, "created_at": VOTE_210_CREATED_AT}]}, ), # Fetch the first page of comments for post 3 ( @@ -960,7 +979,44 @@ def run_incremental_parent_state_test( ), # Fetch the first page of votes for comment 30 of post 3 ( - f"https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time={PARTITION_SYNC_START_TIME}", + f"https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time={LOOKBACK_DATE}", + {"votes": [{"id": 300, "comment_id": 30, "created_at": VOTE_300_CREATED_AT}]}, + ), + # Requests with intermediate states + # Fetch votes for comment 10 of post 1 + ( + f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={VOTE_100_CREATED_AT}", + { + "votes": [{"id": 100, "comment_id": 10, "created_at": VOTE_100_CREATED_AT}], + }, + ), + # Fetch votes for comment 11 of post 1 + ( + f"https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time={VOTE_111_CREATED_AT}", + { + "votes": [{"id": 111, "comment_id": 11, "created_at": VOTE_111_CREATED_AT}], + }, + ), + # Fetch votes for comment 12 of post 1 + ( + f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={VOTE_111_CREATED_AT}", + { + "votes": [], + }, + ), + # Fetch votes for comment 20 of post 2 + ( + f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={VOTE_200_CREATED_AT}", + {"votes": [{"id": 200, "comment_id": 20, "created_at": VOTE_200_CREATED_AT}]}, + ), + # Fetch votes for comment 21 of post 2 + ( + f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={VOTE_210_CREATED_AT}", + {"votes": [{"id": 210, "comment_id": 21, "created_at": VOTE_210_CREATED_AT}]}, + ), + # Fetch votes for comment 30 of post 3 + ( + f"https://api.example.com/community/posts/3/comments/30/votes?per_page=100&start_time={VOTE_300_CREATED_AT}", {"votes": [{"id": 300, "comment_id": 30, "created_at": VOTE_300_CREATED_AT}]}, ), ], @@ -993,8 +1049,8 @@ def run_incremental_parent_state_test( { "comment_id": 21, "comment_updated_at": COMMENT_21_UPDATED_AT, - "created_at": VOTE_201_CREATED_AT, - "id": 201, + "created_at": VOTE_210_CREATED_AT, + "id": 210, }, { "comment_id": 30, @@ -1003,8 +1059,40 @@ def run_incremental_parent_state_test( "id": 300, }, ], + # Number of intermediate states - 6 as number of parent partitions + 6, # Initial state - {"created_at": PARTITION_SYNC_START_TIME}, + { + "parent_state": { + "post_comments": { + "states": [ + { + "partition": {"id": 1, "parent_slice": {}}, + "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, + } + ], + "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, + } + }, + "state": {"created_at": INITIAL_GLOBAL_CURSOR}, + "states": [ + { + "partition": { + "id": 10, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_10_CURSOR}, + }, + { + "partition": { + "id": 11, + "parent_slice": {"id": 1, "parent_slice": {}}, + }, + "cursor": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, + }, + ], + "lookback_window": 86400, + }, # Expected state { "state": {"created_at": VOTE_100_CREATED_AT}, @@ -1044,7 +1132,7 @@ def run_incremental_parent_state_test( }, { "partition": {"id": 12, "parent_slice": {"id": 1, "parent_slice": {}}}, - "cursor": {"created_at": PARTITION_SYNC_START_TIME}, + "cursor": {"created_at": LOOKBACK_DATE}, }, { "partition": {"id": 20, "parent_slice": {"id": 2, "parent_slice": {}}}, @@ -1052,7 +1140,7 @@ def run_incremental_parent_state_test( }, { "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": VOTE_201_CREATED_AT}, + "cursor": {"created_at": VOTE_210_CREATED_AT}, }, { "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, @@ -1064,12 +1152,19 @@ def run_incremental_parent_state_test( ], ) def test_incremental_parent_state( - test_name, manifest, mock_requests, expected_records, initial_state, expected_state + test_name, + manifest, + mock_requests, + expected_records, + num_intermediate_states, + initial_state, + expected_state, ): run_incremental_parent_state_test( manifest, mock_requests, expected_records, + num_intermediate_states, initial_state, [expected_state], ) @@ -1178,7 +1273,7 @@ def test_incremental_parent_state( ( f"https://api.example.com/community/posts/2/comments/21/votes" f"?per_page=100&start_time={PARTITION_SYNC_START_TIME}", - {"votes": [{"id": 201, "comment_id": 21, "created_at": VOTE_201_CREATED_AT}]}, + {"votes": [{"id": 210, "comment_id": 21, "created_at": VOTE_210_CREATED_AT}]}, ), # Fetch the first page of comments for post 3 ( @@ -1221,8 +1316,8 @@ def test_incremental_parent_state( { "comment_id": 21, "comment_updated_at": COMMENT_21_UPDATED_AT, - "created_at": VOTE_201_CREATED_AT, - "id": 201, + "created_at": VOTE_210_CREATED_AT, + "id": 210, }, { "comment_id": 30, @@ -1278,7 +1373,7 @@ def test_incremental_parent_state( }, { "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": VOTE_201_CREATED_AT}, + "cursor": {"created_at": VOTE_210_CREATED_AT}, }, { "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, @@ -1330,82 +1425,6 @@ def test_incremental_parent_state_migration( f"&start_time={PARENT_POSTS_CURSOR}&page=2", {"posts": []}, ), - # Fetch the first page of comments for post 1 - ( - "https://api.example.com/community/posts/1/comments?per_page=100", - { - "comments": [], - "next_page": ( - "https://api.example.com/community/posts/1/comments?per_page=100&page=2" - ), - }, - ), - # Fetch the second page of comments for post 1 - ( - "https://api.example.com/community/posts/1/comments?per_page=100&page=2", - {"comments": []}, - ), - # Fetch the first page of votes for comment 10 of post 1 - ( - f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_10_CURSOR}", - { - "votes": [], - "next_page": ( - "https://api.example.com/community/posts/1/comments/10/votes" - f"?per_page=100&page=2&start_time={START_DATE}" - ), - }, - ), - # Fetch the second page of votes for comment 10 of post 1 - ( - f"https://api.example.com/community/posts/1/comments/10/votes?per_page=100&page=2&start_time={START_DATE}", - {"votes": []}, - ), - # Fetch the first page of votes for comment 11 of post 1 - ( - f"https://api.example.com/community/posts/1/comments/11/votes?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", - {"votes": []}, - ), - # Fetch the first page of votes for comment 12 of post 1 - ( - f"https://api.example.com/community/posts/1/comments/12/votes?per_page=100&start_time={START_DATE}", - {"votes": []}, - ), - # Fetch the first page of comments for post 2 - ( - "https://api.example.com/community/posts/2/comments?per_page=100", - { - "comments": [], - "next_page": ( - "https://api.example.com/community/posts/2/comments?per_page=100&page=2" - ), - }, - ), - # Fetch the second page of comments for post 2 - ( - "https://api.example.com/community/posts/2/comments?per_page=100&page=2", - {"comments": []}, - ), - # Fetch the first page of votes for comment 20 of post 2 - ( - f"https://api.example.com/community/posts/2/comments/20/votes?per_page=100&start_time={START_DATE}", - {"votes": []}, - ), - # Fetch the first page of votes for comment 21 of post 2 - ( - f"https://api.example.com/community/posts/2/comments/21/votes?per_page=100&start_time={START_DATE}", - {"votes": []}, - ), - # Fetch the first page of comments for post 3 - ( - "https://api.example.com/community/posts/3/comments?per_page=100", - {"comments": []}, - ), - # Fetch the first page of votes for comment 30 of post 3 - ( - "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", - {"votes": []}, - ), ], # Expected records (empty) [], @@ -1605,7 +1624,8 @@ def test_incremental_parent_state_no_slices( ), # Fetch the first page of votes for comment 30 of post 3 ( - "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + f"https://api.example.com/community/posts/3/comments/30/votes" + f"?per_page=100&start_time={INITIAL_STATE_PARTITION_11_CURSOR}", {"votes": []}, ), ], @@ -1821,7 +1841,7 @@ def test_incremental_parent_state_no_records( ( f"https://api.example.com/community/posts/2/comments/21/votes" f"?per_page=100&start_time={LOOKBACK_DATE}", - {"votes": [{"id": 201, "comment_id": 21, "created_at": VOTE_201_CREATED_AT}]}, + {"votes": [{"id": 210, "comment_id": 21, "created_at": VOTE_210_CREATED_AT}]}, ), # Fetch the first page of comments for post 3 ( @@ -1830,7 +1850,8 @@ def test_incremental_parent_state_no_records( ), # Fetch the first page of votes for comment 30 of post 3 ( - "https://api.example.com/community/posts/3/comments/30/votes?per_page=100", + f"https://api.example.com/community/posts/3/comments/30/votes" + f"?per_page=100&start_time={LOOKBACK_DATE}", {"votes": [{"id": 300, "comment_id": 30, "created_at": VOTE_300_CREATED_AT}]}, ), ], @@ -1857,8 +1878,8 @@ def test_incremental_parent_state_no_records( { "comment_id": 21, "comment_updated_at": COMMENT_21_UPDATED_AT, - "created_at": VOTE_201_CREATED_AT, - "id": 201, + "created_at": VOTE_210_CREATED_AT, + "id": 210, }, { "comment_id": 30, @@ -1934,7 +1955,7 @@ def test_incremental_parent_state_no_records( }, { "partition": {"id": 21, "parent_slice": {"id": 2, "parent_slice": {}}}, - "cursor": {"created_at": VOTE_201_CREATED_AT}, + "cursor": {"created_at": VOTE_210_CREATED_AT}, }, { "partition": {"id": 30, "parent_slice": {"id": 3, "parent_slice": {}}}, @@ -2122,39 +2143,39 @@ def test_incremental_substream_error( [ # Fetch the first page of comments for post 1 ( - "https://api.example.com/community/posts/1/comments?per_page=100", + "https://api.example.com/community/posts/1/comments?per_page=100&start_time=2024-01-24T00:00:00Z", { "comments": [ {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, ], - "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2&start_time=2024-01-24T00:00:00Z", }, ), # Fetch the second page of comments for post 1 ( - "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "https://api.example.com/community/posts/1/comments?per_page=100&page=2&start_time=2024-01-24T00:00:00Z", {"comments": [{"id": 12, "post_id": 1, "updated_at": "2024-01-23T00:00:00Z"}]}, ), # Fetch the first page of comments for post 2 ( - "https://api.example.com/community/posts/2/comments?per_page=100", + "https://api.example.com/community/posts/2/comments?per_page=100&start_time=2024-01-21T05:00:00Z", { "comments": [ {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} ], - "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2&start_time=2024-01-21T05:00:00Z", }, ), # Fetch the second page of comments for post 2 ( - "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "https://api.example.com/community/posts/2/comments?per_page=100&page=2&start_time=2024-01-21T05:00:00Z", {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, ), # Fetch the first page of comments for post 3 ( - "https://api.example.com/community/posts/3/comments?per_page=100", + "https://api.example.com/community/posts/3/comments?per_page=100&start_time=2024-01-08T00:00:00Z", {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, ), ], @@ -2219,39 +2240,39 @@ def test_incremental_list_partition_router( [ # Fetch the first page of comments for post 1 ( - "https://api.example.com/community/posts/1/comments?per_page=100", + "https://api.example.com/community/posts/1/comments?per_page=100&start_time=2024-01-20T00:00:00Z", { "comments": [ {"id": 9, "post_id": 1, "updated_at": "2023-01-01T00:00:00Z"}, {"id": 10, "post_id": 1, "updated_at": "2024-01-25T00:00:00Z"}, {"id": 11, "post_id": 1, "updated_at": "2024-01-24T00:00:00Z"}, ], - "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "next_page": "https://api.example.com/community/posts/1/comments?per_page=100&page=2&start_time=2024-01-20T00:00:00Z", }, ), # Error response for the second page of comments for post 1 ( - "https://api.example.com/community/posts/1/comments?per_page=100&page=2", + "https://api.example.com/community/posts/1/comments?per_page=100&page=2&start_time=2024-01-20T00:00:00Z", None, # Simulate a network error or an empty response ), # Fetch the first page of comments for post 2 ( - "https://api.example.com/community/posts/2/comments?per_page=100", + "https://api.example.com/community/posts/2/comments?per_page=100&start_time=2024-01-21T05:00:00Z", { "comments": [ {"id": 20, "post_id": 2, "updated_at": "2024-01-22T00:00:00Z"} ], - "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "next_page": "https://api.example.com/community/posts/2/comments?per_page=100&page=2&start_time=2024-01-21T05:00:00Z", }, ), # Fetch the second page of comments for post 2 ( - "https://api.example.com/community/posts/2/comments?per_page=100&page=2", + "https://api.example.com/community/posts/2/comments?per_page=100&page=2&start_time=2024-01-21T05:00:00Z", {"comments": [{"id": 21, "post_id": 2, "updated_at": "2024-01-21T00:00:00Z"}]}, ), # Fetch the first page of comments for post 3 ( - "https://api.example.com/community/posts/3/comments?per_page=100", + "https://api.example.com/community/posts/3/comments?per_page=100&start_time=2024-01-08T00:00:00Z", {"comments": [{"id": 30, "post_id": 3, "updated_at": "2024-01-09T00:00:00Z"}]}, ), ], @@ -2271,7 +2292,7 @@ def test_incremental_list_partition_router( "partition": {"id": "1"}, }, { - "cursor": {"updated_at": "2024-01-22T00:00:00Z"}, + "cursor": {"updated_at": "2024-01-21T05:00:00Z"}, "partition": {"id": "2"}, }, ],