diff --git a/airbyte_cdk/legacy/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/legacy/sources/declarative/incremental/per_partition_cursor.py index 57c7fd21b..54060bd94 100644 --- a/airbyte_cdk/legacy/sources/declarative/incremental/per_partition_cursor.py +++ b/airbyte_cdk/legacy/sources/declarative/incremental/per_partition_cursor.py @@ -146,8 +146,10 @@ def set_initial_state(self, stream_state: StreamState) -> None: 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) + # We used to set the parent state through this method but since moving the SubstreamPartitionRouter to the + # Concurrent CDK/AbstractStream, the state is passed at the __init__ stage and this does not need to be called. + # We are still keeping this line as a comment to be explicit about the past behavior. + # self._partition_router.set_initial_state(stream_state) def observe(self, stream_slice: StreamSlice, record: Record) -> None: self._cursor_per_partition[self._to_partition_key(stream_slice.partition)].observe( diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 16ff94abf..d4e69df78 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -703,7 +703,7 @@ def _group_streams( stream_slicer=declarative_stream.retriever.stream_slicer, slice_limit=self._limits.max_slices if self._limits - else None, # technically not needed because create_declarative_stream() -> create_simple_retriever() will apply the decorator. But for consistency and depending how we build create_default_stream, this may be needed later + else None, # technically not needed because create_default_stream() -> create_simple_retriever() will apply the decorator. But for consistency and depending how we build create_default_stream, this may be needed later ) else: if ( @@ -772,7 +772,7 @@ def _group_streams( declarative_stream.retriever.stream_slicer, slice_limit=self._limits.max_slices if self._limits - else None, # technically not needed because create_declarative_stream() -> create_simple_retriever() will apply the decorator. But for consistency and depending how we build create_default_stream, this may be needed later + else None, # technically not needed because create_default_stream() -> create_simple_retriever() will apply the decorator. But for consistency and depending how we build create_default_stream, this may be needed later ) final_state_cursor = FinalStateCursor( diff --git a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py index 2a7cfd1d3..41ee09e0b 100644 --- a/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/concurrent_partition_cursor.py @@ -11,6 +11,13 @@ from datetime import timedelta from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional +from airbyte_cdk.models import ( + AirbyteStateBlob, + AirbyteStateMessage, + AirbyteStateType, + AirbyteStreamState, + StreamDescriptor, +) from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( Timer, @@ -48,7 +55,7 @@ class ConcurrentPerPartitionCursor(Cursor): Manages state per partition when a stream has many partitions, preventing data loss or duplication. Attributes: - DEFAULT_MAX_PARTITIONS_NUMBER (int): Maximum number of partitions to retain in memory (default is 10,000). + DEFAULT_MAX_PARTITIONS_NUMBER (int): Maximum number of partitions to retain in memory (default is 10,000). This limit needs to be higher than the number of threads we might enqueue (which is represented by ThreadPoolManager.DEFAULT_MAX_QUEUE_SIZE). If not, we could have partitions that have been generated and submitted to the ThreadPool but got deleted from the ConcurrentPerPartitionCursor and when closing them, it will generate KeyError. - **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. @@ -128,6 +135,7 @@ def __init__( # FIXME this is a temporary field the time of the migration from declarative cursors to concurrent ones self._attempt_to_create_cursor_if_not_provided = attempt_to_create_cursor_if_not_provided + self._synced_some_data = False @property def cursor_field(self) -> CursorField: @@ -168,8 +176,8 @@ def close_partition(self, partition: Partition) -> None: with self._lock: self._semaphore_per_partition[partition_key].acquire() if not self._use_global_cursor: - self._cursor_per_partition[partition_key].close_partition(partition=partition) cursor = self._cursor_per_partition[partition_key] + cursor.close_partition(partition=partition) if ( partition_key in self._partitions_done_generating_stream_slices and self._semaphore_per_partition[partition_key]._value == 0 @@ -213,8 +221,10 @@ def ensure_at_least_one_state_emitted(self) -> None: 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() + if self._synced_some_data: + # we only update those if we actually synced some data + 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(throttle=False) @@ -422,9 +432,6 @@ def _set_initial_state(self, stream_state: StreamState) -> None: 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) - def _set_global_state(self, stream_state: Mapping[str, Any]) -> None: """ Initializes the global cursor state from the provided stream state. @@ -458,6 +465,7 @@ def observe(self, record: Record) -> None: except ValueError: return + self._synced_some_data = True record_cursor = self._connector_state_converter.output_format( self._connector_state_converter.parse_value(record_cursor_value) ) @@ -541,3 +549,45 @@ def _get_cursor(self, record: Record) -> ConcurrentCursor: def limit_reached(self) -> bool: return self._number_of_partitions > self.SWITCH_TO_GLOBAL_LIMIT + + @staticmethod + def get_parent_state( + stream_state: Optional[StreamState], parent_stream_name: str + ) -> Optional[AirbyteStateMessage]: + if not stream_state: + return None + + if "parent_state" not in stream_state: + logger.warning( + f"Trying to get_parent_state for stream `{parent_stream_name}` when there are not parent state in the state" + ) + return None + elif parent_stream_name not in stream_state["parent_state"]: + logger.info( + f"Could not find parent state for stream `{parent_stream_name}`. On parents available are {list(stream_state['parent_state'].keys())}" + ) + return None + + return AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(parent_stream_name, None), + stream_state=AirbyteStateBlob(stream_state["parent_state"][parent_stream_name]), + ), + ) + + @staticmethod + def get_global_state( + stream_state: Optional[StreamState], parent_stream_name: str + ) -> Optional[AirbyteStateMessage]: + return ( + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(parent_stream_name, None), + stream_state=AirbyteStateBlob(stream_state["state"]), + ), + ) + if stream_state and "state" in stream_state + else None + ) diff --git a/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py b/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py index 602ea051b..21733f94d 100644 --- a/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py +++ b/airbyte_cdk/sources/declarative/incremental/global_substream_cursor.py @@ -192,8 +192,10 @@ def set_initial_state(self, stream_state: StreamState) -> None: # Example: {"global_state_format_key": "global_state_format_value"} self._stream_cursor.set_initial_state(stream_state) - # Set parent state for partition routers based on parent streams - self._partition_router.set_initial_state(stream_state) + # We used to set the parent state through this method but since moving the SubstreamPartitionRouter to the + # Concurrent CDK/AbstractStream, the state is passed at the __init__ stage and this does not need to be called. + # We are still keeping this line as a comment to be explicit about the past behavior. + # self._partition_router.set_initial_state(stream_state) def _inject_lookback_into_stream_cursor(self, lookback_window: int) -> None: """ 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 649e66b44..105f472de 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -33,7 +33,15 @@ from airbyte_cdk.connector_builder.models import ( LogMessage as ConnectorBuilderLogMessage, ) -from airbyte_cdk.models import FailureType, Level +from airbyte_cdk.models import ( + AirbyteStateBlob, + AirbyteStateMessage, + AirbyteStateType, + AirbyteStreamState, + FailureType, + Level, + StreamDescriptor, +) from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.declarative.async_job.job_orchestrator import AsyncJobOrchestrator from airbyte_cdk.sources.declarative.async_job.job_tracker import JobTracker @@ -90,6 +98,7 @@ RecordSelector, ResponseToFileExtractor, ) +from airbyte_cdk.sources.declarative.extractors.record_extractor import RecordExtractor from airbyte_cdk.sources.declarative.extractors.record_filter import ( ClientSideIncrementalRecordFilterDecorator, ) @@ -98,7 +107,6 @@ ConcurrentPerPartitionCursor, CursorFactory, DatetimeBasedCursor, - DeclarativeCursor, GlobalSubstreamCursor, PerPartitionWithGlobalCursor, ) @@ -500,8 +508,11 @@ InterpolatedRequestOptionsProvider, RequestOptionsProvider, ) +from airbyte_cdk.sources.declarative.requesters.request_options.per_partition_request_option_provider import ( + PerPartitionRequestOptionsProvider, +) from airbyte_cdk.sources.declarative.requesters.request_path import RequestPath -from airbyte_cdk.sources.declarative.requesters.requester import HttpMethod +from airbyte_cdk.sources.declarative.requesters.requester import HttpMethod, Requester from airbyte_cdk.sources.declarative.resolvers import ( ComponentMappingDefinition, ConfigComponentsResolver, @@ -583,6 +594,7 @@ MessageRepository, NoopMessageRepository, ) +from airbyte_cdk.sources.message.repository import StateFilteringMessageRepository from airbyte_cdk.sources.streams.call_rate import ( APIBudget, FixedWindowCallRatePolicy, @@ -630,6 +642,7 @@ SchemaNormalizationModel.None_: TransformConfig.NoTransform, SchemaNormalizationModel.Default: TransformConfig.DefaultSchemaNormalization, } +_NO_STREAM_SLICING = SinglePartitionRouter(parameters={}) # Ideally this should use the value defined in ConcurrentDeclarativeSource, but # this would be a circular import @@ -702,7 +715,7 @@ def _init_mappings(self) -> None: CustomValidationStrategyModel: self.create_custom_component, CustomConfigTransformationModel: self.create_custom_component, DatetimeBasedCursorModel: self.create_datetime_based_cursor, - DeclarativeStreamModel: self.create_declarative_stream, + DeclarativeStreamModel: self.create_default_stream, DefaultErrorHandlerModel: self.create_default_error_handler, DefaultPaginatorModel: self.create_default_paginator, DpathExtractorModel: self.create_dpath_extractor, @@ -739,7 +752,7 @@ def _init_mappings(self) -> None: OAuthAuthenticatorModel: self.create_oauth_authenticator, OffsetIncrementModel: self.create_offset_increment, PageIncrementModel: self.create_page_increment, - ParentStreamConfigModel: self.create_parent_stream_config, + ParentStreamConfigModel: self.create_parent_stream_config_with_substream_wrapper, PredicateValidatorModel: self.create_predicate_validator, PropertiesFromEndpointModel: self.create_properties_from_endpoint, PropertyChunkingModel: self.create_property_chunking, @@ -1291,19 +1304,20 @@ def create_concurrent_cursor_from_datetime_based_cursor( f"Expected {model_type.__name__} component, but received {datetime_based_cursor_model.__class__.__name__}" ) + model_parameters = datetime_based_cursor_model.parameters or {} interpolated_cursor_field = InterpolatedString.create( datetime_based_cursor_model.cursor_field, - parameters=datetime_based_cursor_model.parameters or {}, + parameters=model_parameters, ) 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 {}, + parameters=model_parameters, ) interpolated_partition_field_end = InterpolatedString.create( datetime_based_cursor_model.partition_field_end or "end_time", - parameters=datetime_based_cursor_model.parameters or {}, + parameters=model_parameters, ) slice_boundary_fields = ( @@ -1323,7 +1337,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( interpolated_lookback_window = ( InterpolatedString.create( datetime_based_cursor_model.lookback_window, - parameters=datetime_based_cursor_model.parameters or {}, + parameters=model_parameters, ) if datetime_based_cursor_model.lookback_window else None @@ -1409,7 +1423,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( interpolated_step = ( InterpolatedString.create( datetime_based_cursor_model.step, - parameters=datetime_based_cursor_model.parameters or {}, + parameters=model_parameters, ) if datetime_based_cursor_model.step else None @@ -1426,7 +1440,7 @@ def create_concurrent_cursor_from_datetime_based_cursor( # object which we want to keep agnostic of being low-code target = InterpolatedString( string=datetime_based_cursor_model.clamping.target, - parameters=datetime_based_cursor_model.parameters or {}, + parameters=model_parameters, ) evaluated_target = target.eval(config=config) match evaluated_target: @@ -1603,6 +1617,10 @@ def create_concurrent_cursor_from_perpartition_cursor( interpolated_cursor_field = InterpolatedString.create( datetime_based_cursor_model.cursor_field, + # FIXME the interfaces of the concurrent cursor are kind of annoying as they take a `ComponentDefinition` instead of the actual model. This was done because the ConcurrentDeclarativeSource didn't have access to the models [here for example](https://github.com/airbytehq/airbyte-python-cdk/blob/f525803b3fec9329e4cc8478996a92bf884bfde9/airbyte_cdk/sources/declarative/concurrent_declarative_source.py#L354C54-L354C91). So now we have two cases: + # * The ComponentDefinition comes from model.__dict__ in which case we have `parameters` + # * The ComponentDefinition comes from the manifest as a dict in which case we have `$parameters` + # We should change those interfaces to use the model once we clean up the code in CDS at which point the parameter propagation should happen as part of the ModelToComponentFactory. parameters=datetime_based_cursor_model.parameters or {}, ) cursor_field = CursorField(interpolated_cursor_field.eval(config=config)) @@ -1634,7 +1652,7 @@ def create_concurrent_cursor_from_perpartition_cursor( stream_namespace=stream_namespace, config=config, message_repository=NoopMessageRepository(), - stream_state_migrations=stream_state_migrations, + # stream_state_migrations=stream_state_migrations, # FIXME is it expected to run migration on per partition state too? ) ) @@ -1730,7 +1748,11 @@ def create_custom_component(self, model: Any, config: Config, **kwargs: Any) -> if self._is_component(model_value): model_args[model_field] = self._create_nested_component( - model, model_field, model_value, config + model, + model_field, + model_value, + config, + **kwargs, ) elif isinstance(model_value, list): vals = [] @@ -1742,7 +1764,15 @@ def create_custom_component(self, model: Any, config: Config, **kwargs: Any) -> if derived_type: v["type"] = derived_type if self._is_component(v): - vals.append(self._create_nested_component(model, model_field, v, config)) + vals.append( + self._create_nested_component( + model, + model_field, + v, + config, + **kwargs, + ) + ) else: vals.append(v) model_args[model_field] = vals @@ -1832,7 +1862,7 @@ def _extract_missing_parameters(error: TypeError) -> List[str]: return [] def _create_nested_component( - self, model: Any, model_field: str, model_value: Any, config: Config + self, model: Any, model_field: str, model_value: Any, config: Config, **kwargs: Any ) -> Any: type_name = model_value.get("type", None) if not type_name: @@ -1857,8 +1887,11 @@ def _create_nested_component( for kwarg in constructor_kwargs if kwarg in model_parameters } + matching_kwargs = { + kwarg: kwargs[kwarg] for kwarg in constructor_kwargs if kwarg in kwargs + } return self._create_component_from_model( - model=parsed_model, config=config, **matching_parameters + model=parsed_model, config=config, **(matching_parameters | matching_kwargs) ) except TypeError as error: missing_parameters = self._extract_missing_parameters(error) @@ -1942,13 +1975,17 @@ def create_datetime_based_cursor( parameters=model.parameters or {}, ) - def create_declarative_stream( + def create_default_stream( self, model: DeclarativeStreamModel, config: Config, is_parent: bool = False, **kwargs: Any ) -> Union[DeclarativeStream, AbstractStream]: primary_key = model.primary_key.__root__ if model.primary_key else None + partition_router = self._build_stream_slicer_from_partition_router( + model.retriever, config, stream_name=model.name + ) + concurrent_cursor = self._build_concurrent_cursor(model, partition_router, config) if model.incremental_sync and isinstance(model.incremental_sync, DatetimeBasedCursorModel): - cursor_model = model.incremental_sync + cursor_model: DatetimeBasedCursorModel = model.incremental_sync end_time_option = ( self._create_component_from_model( @@ -1965,17 +2002,29 @@ def create_declarative_stream( else None ) - request_options_provider = DatetimeBasedRequestOptionsProvider( + datetime_request_options_provider = DatetimeBasedRequestOptionsProvider( start_time_option=start_time_option, end_time_option=end_time_option, - partition_field_start=cursor_model.partition_field_end, + partition_field_start=cursor_model.partition_field_start, partition_field_end=cursor_model.partition_field_end, config=config, parameters=model.parameters or {}, ) + request_options_provider = ( + datetime_request_options_provider + if not isinstance(concurrent_cursor, ConcurrentPerPartitionCursor) + else PerPartitionRequestOptionsProvider( + partition_router, datetime_request_options_provider + ) + ) elif model.incremental_sync and isinstance( model.incremental_sync, IncrementingCountCursorModel ): + if isinstance(concurrent_cursor, ConcurrentPerPartitionCursor): + raise ValueError( + "PerPartition does not support per partition states because switching to global state is time based" + ) + cursor_model: IncrementingCountCursorModel = model.incremental_sync # type: ignore start_time_option = ( @@ -2013,22 +2062,19 @@ def create_declarative_stream( model=model.file_uploader, config=config ) - # When constructing a declarative stream, we assemble the incremental_sync component and retriever's partition_router field - # components if they exist into a single CartesianProductStreamSlicer. This is then passed back as an argument when constructing the - # Retriever. This is done in the declarative stream not the retriever to support custom retrievers. The custom create methods in - # the factory only support passing arguments to the component constructors, whereas this performs a merge of all slicers into one. - combined_slicers = self._merge_stream_slicers(model=model, config=config) - partition_router = self._build_stream_slicer_from_partition_router( - model.retriever, config, stream_name=model.name + stream_slicer: ConcurrentStreamSlicer = ( + partition_router + if isinstance(concurrent_cursor, FinalStateCursor) + else concurrent_cursor ) - concurrent_cursor = self._build_concurrent_cursor(model, partition_router, config) retriever = self._create_component_from_model( model=model.retriever, config=config, name=model.name, primary_key=primary_key, - stream_slicer=combined_slicers, request_options_provider=request_options_provider, + stream_slicer=stream_slicer, + partition_router=partition_router, stop_condition_cursor=concurrent_cursor if self._is_stop_condition_on_cursor(model) else None, @@ -2039,6 +2085,8 @@ def create_declarative_stream( file_uploader=file_uploader, incremental_sync=model.incremental_sync, ) + if isinstance(retriever, AsyncRetriever): + stream_slicer = retriever.stream_slicer schema_loader: Union[ CompositeSchemaLoader, @@ -2066,89 +2114,27 @@ def create_declarative_stream( options["name"] = model.name schema_loader = DefaultSchemaLoader(config=config, parameters=options) - if ( - ( - isinstance(combined_slicers, PartitionRouter) - or isinstance(concurrent_cursor, ConcurrentCursor) - ) - and not self._emit_connector_builder_messages - and not is_parent - ): - # We are starting to migrate streams to instantiate directly the DefaultStream instead of instantiating the - # DeclarativeStream and assembling the DefaultStream from that. The plan is the following: - # * Streams without partition router nor cursors and streams with only partition router. This is the `isinstance(combined_slicers, PartitionRouter)` condition as the first kind with have a SinglePartitionRouter - # * Streams without partition router but with cursor. This is the `isinstance(concurrent_cursor, ConcurrentCursor)` condition - # * Streams with both partition router and cursor - # We specifically exclude parent streams here because SubstreamPartitionRouter has not been updated yet - # We specifically exclude Connector Builder stuff for now as Brian is working on this anyway - - stream_name = model.name or "" - stream_slicer: ConcurrentStreamSlicer = ( - concurrent_cursor if concurrent_cursor else SinglePartitionRouter(parameters={}) - ) - cursor: Cursor = FinalStateCursor(stream_name, None, self._message_repository) - if isinstance(retriever, AsyncRetriever): - # The AsyncRetriever only ever worked with a cursor from the concurrent package. Hence, the method - # `_build_incremental_cursor` which we would usually think would return only declarative stuff has a - # special clause and return a concurrent cursor. This stream slicer is passed to AsyncRetriever when - # built because the async retriever has a specific partition router which relies on this stream slicer. - # We can't re-use `concurrent_cursor` because it is a different instance than the one passed in - # AsyncJobPartitionRouter. - stream_slicer = retriever.stream_slicer - if isinstance(combined_slicers, Cursor): - cursor = combined_slicers - elif isinstance(combined_slicers, PartitionRouter): - stream_slicer = combined_slicers - elif concurrent_cursor: - cursor = concurrent_cursor - - # FIXME to be removed once we migrate everything to DefaultStream - if isinstance(retriever, SimpleRetriever): - # 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( + stream_name = model.name or "" + return DefaultStream( + partition_generator=StreamSlicerPartitionGenerator( DeclarativePartitionFactory( stream_name, schema_loader, retriever, self._message_repository, ), - stream_slicer=stream_slicer, - ) - return DefaultStream( - partition_generator=partition_generator, - name=stream_name, - json_schema=schema_loader.get_json_schema, - primary_key=get_primary_key_from_stream(primary_key), - cursor_field=cursor.cursor_field.cursor_field_key - if hasattr(cursor, "cursor_field") - else "", # FIXME we should have the cursor field has part of the interface of cursor, - logger=logging.getLogger(f"airbyte.{stream_name}"), - # FIXME this is a breaking change compared to the old implementation which used the source name instead - cursor=cursor, - supports_file_transfer=hasattr(model, "file_uploader") - and bool(model.file_uploader), - ) - - cursor_field = model.incremental_sync.cursor_field if model.incremental_sync else None - if model.state_migrations: - state_transformations = [ - self._create_component_from_model(state_migration, config, declarative_stream=model) - for state_migration in model.state_migrations - ] - else: - state_transformations = [] - return DeclarativeStream( - name=model.name or "", - primary_key=primary_key, - retriever=retriever, - schema_loader=schema_loader, - stream_cursor_field=cursor_field or "", - state_migrations=state_transformations, - config=config, - parameters=model.parameters or {}, + stream_slicer, + slice_limit=self._limit_slices_fetched, + ), + name=stream_name, + json_schema=schema_loader.get_json_schema, + primary_key=get_primary_key_from_stream(primary_key), + cursor_field=concurrent_cursor.cursor_field.cursor_field_key + if hasattr(concurrent_cursor, "cursor_field") + else "", # FIXME we should have the cursor field has part of the interface of cursor, + logger=logging.getLogger(f"airbyte.{stream_name}"), + cursor=concurrent_cursor, + supports_file_transfer=hasattr(model, "file_uploader") and bool(model.file_uploader), ) def _is_stop_condition_on_cursor(self, model: DeclarativeStreamModel) -> bool: @@ -2197,86 +2183,15 @@ def _build_stream_slicer_from_partition_router( ) return SinglePartitionRouter(parameters={}) - def _build_incremental_cursor( - self, - model: DeclarativeStreamModel, - stream_slicer: Optional[PartitionRouter], - config: Config, - ) -> Optional[StreamSlicer]: - state_transformations = ( - [ - self._create_component_from_model(state_migration, config, declarative_stream=model) - for state_migration in model.state_migrations - ] - if model.state_migrations - else [] - ) - - if model.incremental_sync and ( - stream_slicer and not isinstance(stream_slicer, SinglePartitionRouter) - ): - if model.retriever.type == "AsyncRetriever": - stream_name = model.name or "" - stream_namespace = None - stream_state = self._connector_state_manager.get_stream_state( - stream_name, stream_namespace - ) - - return self.create_concurrent_cursor_from_perpartition_cursor( # type: ignore # This is a known issue that we are creating and returning a ConcurrentCursor which does not technically implement the (low-code) StreamSlicer. However, (low-code) StreamSlicer and ConcurrentCursor both implement StreamSlicer.stream_slices() which is the primary method needed for checkpointing - state_manager=self._connector_state_manager, - model_type=DatetimeBasedCursorModel, - component_definition=model.incremental_sync.__dict__, - stream_name=stream_name, - stream_namespace=stream_namespace, - config=config or {}, - stream_state=stream_state, - stream_state_migrations=state_transformations, - partition_router=stream_slicer, - ) - - incremental_sync_model = model.incremental_sync - cursor_component = self._create_component_from_model( - model=incremental_sync_model, config=config - ) - is_global_cursor = ( - hasattr(incremental_sync_model, "global_substream_cursor") - and incremental_sync_model.global_substream_cursor - ) - - if is_global_cursor: - return GlobalSubstreamCursor( - stream_cursor=cursor_component, partition_router=stream_slicer - ) - return PerPartitionWithGlobalCursor( - cursor_factory=CursorFactory( - lambda: self._create_component_from_model( - model=incremental_sync_model, config=config - ), - ), - partition_router=stream_slicer, - stream_cursor=cursor_component, - ) - elif model.incremental_sync: - if model.retriever.type == "AsyncRetriever": - return self.create_concurrent_cursor_from_datetime_based_cursor( # type: ignore # This is a known issue that we are creating and returning a ConcurrentCursor which does not technically implement the (low-code) StreamSlicer. However, (low-code) StreamSlicer and ConcurrentCursor both implement StreamSlicer.stream_slices() which is the primary method needed for checkpointing - model_type=DatetimeBasedCursorModel, - component_definition=model.incremental_sync.__dict__, - stream_name=model.name or "", - stream_namespace=None, - config=config or {}, - stream_state_migrations=state_transformations, - ) - return self._create_component_from_model(model=model.incremental_sync, config=config) # type: ignore[no-any-return] # Will be created Cursor as stream_slicer_model is model.incremental_sync - return None - def _build_concurrent_cursor( self, model: DeclarativeStreamModel, stream_slicer: Optional[PartitionRouter], config: Config, - ) -> Optional[StreamSlicer]: + ) -> Cursor: + stream_name = model.name or "" stream_state = self._connector_state_manager.get_stream_state( - stream_name=model.name or "", namespace=None + stream_name=stream_name, namespace=None ) if model.state_migrations: @@ -2296,20 +2211,20 @@ def _build_concurrent_cursor( state_manager=self._connector_state_manager, model_type=DatetimeBasedCursorModel, component_definition=model.incremental_sync.__dict__, - stream_name=model.name or "", + stream_name=stream_name, stream_namespace=None, config=config or {}, stream_state=stream_state, stream_state_migrations=state_transformations, partition_router=stream_slicer, - attempt_to_create_cursor_if_not_provided=True, + attempt_to_create_cursor_if_not_provided=True, # FIXME can we remove that now? ) elif model.incremental_sync: if type(model.incremental_sync) == IncrementingCountCursorModel: return self.create_concurrent_cursor_from_incrementing_count_cursor( # type: ignore # This is a known issue that we are creating and returning a ConcurrentCursor which does not technically implement the (low-code) StreamSlicer. However, (low-code) StreamSlicer and ConcurrentCursor both implement StreamSlicer.stream_slices() which is the primary method needed for checkpointing model_type=IncrementingCountCursorModel, component_definition=model.incremental_sync.__dict__, - stream_name=model.name or "", + stream_name=stream_name, stream_namespace=None, config=config or {}, stream_state_migrations=state_transformations, @@ -2318,7 +2233,7 @@ def _build_concurrent_cursor( return self.create_concurrent_cursor_from_datetime_based_cursor( # type: ignore # This is a known issue that we are creating and returning a ConcurrentCursor which does not technically implement the (low-code) StreamSlicer. However, (low-code) StreamSlicer and ConcurrentCursor both implement StreamSlicer.stream_slices() which is the primary method needed for checkpointing model_type=type(model.incremental_sync), component_definition=model.incremental_sync.__dict__, - stream_name=model.name or "", + stream_name=stream_name, stream_namespace=None, config=config or {}, stream_state_migrations=state_transformations, @@ -2328,45 +2243,7 @@ def _build_concurrent_cursor( raise ValueError( f"Incremental sync of type {type(model.incremental_sync)} is not supported" ) - return None - - def _merge_stream_slicers( - self, model: DeclarativeStreamModel, config: Config - ) -> Optional[StreamSlicer]: - retriever_model = model.retriever - - stream_slicer = self._build_stream_slicer_from_partition_router( - retriever_model, config, stream_name=model.name - ) - - if retriever_model.type == "AsyncRetriever": - is_not_datetime_cursor = ( - model.incremental_sync.type != "DatetimeBasedCursor" - if model.incremental_sync - else None - ) - is_partition_router = ( - bool(retriever_model.partition_router) if model.incremental_sync else None - ) - - if is_not_datetime_cursor: - # We are currently in a transition to the Concurrent CDK and AsyncRetriever can only work with the - # support or unordered slices (for example, when we trigger reports for January and February, the report - # in February can be completed first). Once we have support for custom concurrent cursor or have a new - # implementation available in the CDK, we can enable more cursors here. - raise ValueError( - "AsyncRetriever with cursor other than DatetimeBasedCursor is not supported yet." - ) - - if is_partition_router and not stream_slicer: - # Note that this development is also done in parallel to the per partition development which once merged - # we could support here by calling create_concurrent_cursor_from_perpartition_cursor - raise ValueError("Per partition state is not supported yet for AsyncRetriever.") - - if model.incremental_sync: - return self._build_incremental_cursor(model, stream_slicer, config) - - return stream_slicer + return FinalStateCursor(stream_name, None, self._message_repository) def create_default_error_handler( self, model: DefaultErrorHandlerModel, config: Config, **kwargs: Any @@ -2660,7 +2537,9 @@ def create_dynamic_schema_loader( config=config, name=name, primary_key=None, - stream_slicer=self._build_stream_slicer_from_partition_router(model.retriever, config), + partition_router=self._build_stream_slicer_from_partition_router( + model.retriever, config + ), transformations=[], use_cache=True, log_formatter=( @@ -3010,7 +2889,7 @@ def create_page_increment( ) def create_parent_stream_config( - self, model: ParentStreamConfigModel, config: Config, **kwargs: Any + self, model: ParentStreamConfigModel, config: Config, stream_name: str, **kwargs: Any ) -> ParentStreamConfig: declarative_stream = self._create_component_from_model( model.stream, @@ -3263,7 +3142,6 @@ def create_simple_retriever( *, name: str, primary_key: Optional[Union[str, List[str], List[List[str]]]], - stream_slicer: Optional[StreamSlicer], request_options_provider: Optional[RequestOptionsProvider] = None, stop_condition_cursor: Optional[Cursor] = None, client_side_incremental_sync: Optional[Dict[str, Any]] = None, @@ -3276,9 +3154,10 @@ def create_simple_retriever( ] = None, use_cache: Optional[bool] = None, log_formatter: Optional[Callable[[Response], Any]] = None, + partition_router: Optional[PartitionRouter] = None, **kwargs: Any, ) -> SimpleRetriever: - def _get_url() -> str: + def _get_url(req: Requester) -> str: """ Closure to get the URL from the requester. This is used to get the URL in the case of a lazy retriever. This is needed because the URL is not set until the requester is created. @@ -3287,12 +3166,12 @@ def _get_url() -> str: _url: str = ( model.requester.url if hasattr(model.requester, "url") and model.requester.url is not None - else requester.get_url() + else req.get_url(stream_state=None, stream_slice=None, next_page_token=None) ) _url_base: str = ( model.requester.url_base if hasattr(model.requester, "url_base") and model.requester.url_base is not None - else requester.get_url_base() + else req.get_url_base(stream_state=None, stream_slice=None, next_page_token=None) ) return _url or _url_base @@ -3371,36 +3250,18 @@ def _get_url() -> str: config=config, ) - # Define cursor only if per partition or common incremental support is needed - cursor = stream_slicer if isinstance(stream_slicer, DeclarativeCursor) else None - - if ( - not isinstance(stream_slicer, DatetimeBasedCursor) - or type(stream_slicer) is not DatetimeBasedCursor - ): - # 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 - # request_options_provider - request_options_provider = stream_slicer or DefaultRequestOptionsProvider(parameters={}) - elif not request_options_provider: + if not request_options_provider: request_options_provider = DefaultRequestOptionsProvider(parameters={}) - - stream_slicer = stream_slicer or SinglePartitionRouter(parameters={}) - if self._should_limit_slices_fetched(): - stream_slicer = cast( - StreamSlicer, - StreamSlicerTestReadDecorator( - wrapped_slicer=stream_slicer, - maximum_number_of_slices=self._limit_slices_fetched or 5, - ), - ) + if isinstance(request_options_provider, DefaultRequestOptionsProvider) and isinstance( + partition_router, PartitionRouter + ): + request_options_provider = partition_router paginator = ( self._create_component_from_model( model=model.paginator, config=config, - url_base=_get_url(), + url_base=_get_url(requester), extractor_model=model.record_selector.extractor, decoder=decoder, cursor_used_for_stop_condition=stop_condition_cursor or None, @@ -3444,9 +3305,9 @@ def _get_url() -> str: primary_key=primary_key, requester=requester, record_selector=record_selector, - stream_slicer=stream_slicer, + stream_slicer=_NO_STREAM_SLICING, request_option_provider=request_options_provider, - cursor=cursor, + cursor=None, config=config, ignore_stream_slicer_parameters_on_paginated_requests=ignore_stream_slicer_parameters_on_paginated_requests, parameters=model.parameters or {}, @@ -3458,9 +3319,9 @@ def _get_url() -> str: primary_key=primary_key, requester=requester, record_selector=record_selector, - stream_slicer=stream_slicer, + stream_slicer=_NO_STREAM_SLICING, request_option_provider=request_options_provider, - cursor=cursor, + cursor=None, config=config, ignore_stream_slicer_parameters_on_paginated_requests=ignore_stream_slicer_parameters_on_paginated_requests, additional_query_properties=query_properties, @@ -3531,14 +3392,21 @@ def create_state_delegating_stream( f"state_delegating_stream, full_refresh_stream name and incremental_stream must have equal names. Instead has {model.name}, {model.full_refresh_stream.name} and {model.incremental_stream.name}." ) - stream_model = ( + stream_model = self._get_state_delegating_stream_model( + False if has_parent_state is None else has_parent_state, model + ) + + return self._create_component_from_model(stream_model, config=config, **kwargs) # type: ignore[no-any-return] # DeclarativeStream will be created as stream_model is alwyas DeclarativeStreamModel + + def _get_state_delegating_stream_model( + self, has_parent_state: bool, model: StateDelegatingStreamModel + ) -> DeclarativeStreamModel: + return ( model.incremental_stream if self._connector_state_manager.get_stream_state(model.name, None) or has_parent_state else model.full_refresh_stream ) - return self._create_component_from_model(stream_model, config=config, **kwargs) # type: ignore[no-any-return] # Will be created DeclarativeStream as stream_model is stream description - def _create_async_job_status_mapping( self, model: AsyncJobStatusMapModel, config: Config, **kwargs: Any ) -> Mapping[str, AsyncJobStatus]: @@ -3583,12 +3451,14 @@ def create_async_retriever( transformations: List[RecordTransformation], **kwargs: Any, ) -> AsyncRetriever: - def _get_download_retriever() -> SimpleRetriever: + def _get_download_retriever( + requester: Requester, extractor: RecordExtractor, _decoder: Decoder + ) -> SimpleRetriever: # We create a record selector for the download retriever # with no schema normalization and no transformations, neither record filter # as all this occurs in the record_selector of the AsyncRetriever record_selector = RecordSelector( - extractor=download_extractor, + extractor=extractor, name=name, record_filter=None, transformations=[], @@ -3599,7 +3469,7 @@ def _get_download_retriever() -> SimpleRetriever: paginator = ( self._create_component_from_model( model=model.download_paginator, - decoder=decoder, + decoder=_decoder, config=config, url_base="", ) @@ -3608,7 +3478,7 @@ def _get_download_retriever() -> SimpleRetriever: ) return SimpleRetriever( - requester=download_requester, + requester=requester, record_selector=record_selector, primary_key=None, name=name, @@ -3702,7 +3572,9 @@ def _get_job_timeout() -> datetime.timedelta: config=config, name=job_download_components_name, ) - download_retriever = _get_download_retriever() + download_retriever = _get_download_retriever( + download_requester, download_extractor, download_decoder + ) abort_requester = ( self._create_component_from_model( model=model.abort_requester, @@ -3832,7 +3704,7 @@ def create_substream_partition_router( if model.parent_stream_configs: parent_stream_configs.extend( [ - self._create_message_repository_substream_wrapper( + self.create_parent_stream_config_with_substream_wrapper( model=parent_stream_config, config=config, **kwargs ) for parent_stream_config in model.parent_stream_configs @@ -3845,32 +3717,105 @@ def create_substream_partition_router( config=config, ) - def _create_message_repository_substream_wrapper( - self, model: ParentStreamConfigModel, config: Config, **kwargs: Any + def create_parent_stream_config_with_substream_wrapper( + self, model: ParentStreamConfigModel, config: Config, *, stream_name: str, **kwargs: Any ) -> Any: + # getting the parent state + child_state = self._connector_state_manager.get_stream_state(stream_name, None) + + # This flag will be used exclusively for StateDelegatingStream when a parent stream is created + has_parent_state = bool( + self._connector_state_manager.get_stream_state(kwargs.get("stream_name", ""), None) + if model.incremental_dependency + else False + ) + connector_state_manager = self._instantiate_parent_stream_state_manager( + child_state, config, model, has_parent_state + ) + substream_factory = ModelToComponentFactory( + connector_state_manager=connector_state_manager, limit_pages_fetched_per_slice=self._limit_pages_fetched_per_slice, limit_slices_fetched=self._limit_slices_fetched, emit_connector_builder_messages=self._emit_connector_builder_messages, disable_retries=self._disable_retries, disable_cache=self._disable_cache, - message_repository=LogAppenderMessageRepositoryDecorator( - {"airbyte_cdk": {"stream": {"is_substream": True}}, "http": {"is_auxiliary": True}}, - self._message_repository, - self._evaluate_log_level(self._emit_connector_builder_messages), + message_repository=StateFilteringMessageRepository( + LogAppenderMessageRepositoryDecorator( + { + "airbyte_cdk": {"stream": {"is_substream": True}}, + "http": {"is_auxiliary": True}, + }, + self._message_repository, + self._evaluate_log_level(self._emit_connector_builder_messages), + ), ), ) - # This flag will be used exclusively for StateDelegatingStream when a parent stream is created - has_parent_state = bool( - self._connector_state_manager.get_stream_state(kwargs.get("stream_name", ""), None) - if model.incremental_dependency - else False - ) - return substream_factory._create_component_from_model( - model=model, config=config, has_parent_state=has_parent_state, **kwargs + return substream_factory.create_parent_stream_config( + model=model, config=config, stream_name=stream_name, **kwargs ) + def _instantiate_parent_stream_state_manager( + self, + child_state: MutableMapping[str, Any], + config: Config, + model: ParentStreamConfigModel, + has_parent_state: bool, + ) -> ConnectorStateManager: + """ + With DefaultStream, the state needs to be provided during __init__ of the cursor as opposed to the + `set_initial_state` flow that existed for the declarative cursors. This state is taken from + self._connector_state_manager.get_stream_state (`self` being a newly created ModelToComponentFactory to account + for the MessageRepository being different). So we need to pass a ConnectorStateManager to the + ModelToComponentFactory that has the parent states. This method populates this if there is a child state and if + incremental_dependency is set. + """ + if model.incremental_dependency and child_state: + parent_stream_name = model.stream.name or "" + parent_state = ConcurrentPerPartitionCursor.get_parent_state( + child_state, parent_stream_name + ) + + if not parent_state: + # there are two migration cases: state value from child stream or from global state + parent_state = ConcurrentPerPartitionCursor.get_global_state( + child_state, parent_stream_name + ) + + if not parent_state and not isinstance(parent_state, dict): + cursor_values = child_state.values() + if cursor_values: + incremental_sync_model: Union[ + DatetimeBasedCursorModel, + IncrementingCountCursorModel, + CustomIncrementalSyncModel, + ] = ( + model.stream.incremental_sync # type: ignore # if we are there, it is because there is incremental_dependency and therefore there is an incremental_sync on the parent stream + if isinstance(model.stream, DeclarativeStreamModel) + else self._get_state_delegating_stream_model( + has_parent_state, model.stream + ).incremental_sync + ) + cursor_field = InterpolatedString.create( + incremental_sync_model.cursor_field, + parameters=incremental_sync_model.parameters or {}, + ).eval(config) + parent_state = AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor( + name=parent_stream_name, namespace=None + ), + stream_state=AirbyteStateBlob( + {cursor_field: list(cursor_values)[0]} + ), + ), + ) + return ConnectorStateManager([parent_state] if parent_state else []) + + return ConnectorStateManager([]) + @staticmethod def create_wait_time_from_header( model: WaitTimeFromHeaderModel, config: Config, **kwargs: Any @@ -3951,6 +3896,7 @@ def create_http_components_resolver( return HttpComponentsResolver( retriever=retriever, + stream_slicer=self._build_stream_slicer_from_partition_router(model.retriever, config), config=config, components_mapping=components_mapping, parameters=model.parameters or {}, @@ -4176,7 +4122,9 @@ def create_grouping_partition_router( self, model: GroupingPartitionRouterModel, config: Config, **kwargs: Any ) -> GroupingPartitionRouter: underlying_router = self._create_component_from_model( - model=model.underlying_partition_router, config=config + model=model.underlying_partition_router, + config=config, + **kwargs, ) if model.group_size < 1: raise ValueError(f"Group size must be greater than 0, got {model.group_size}") diff --git a/airbyte_cdk/sources/declarative/partition_routers/cartesian_product_stream_slicer.py b/airbyte_cdk/sources/declarative/partition_routers/cartesian_product_stream_slicer.py index db5f5fae7..d7cf20b2c 100644 --- a/airbyte_cdk/sources/declarative/partition_routers/cartesian_product_stream_slicer.py +++ b/airbyte_cdk/sources/declarative/partition_routers/cartesian_product_stream_slicer.py @@ -162,12 +162,6 @@ def stream_slices(self) -> Iterable[StreamSlice]: partition=partition, cursor_slice=cursor_slice, extra_fields=extra_fields ) - def set_initial_state(self, stream_state: StreamState) -> None: - """ - Parent stream states are not supported for cartesian product stream slicer - """ - pass - def get_stream_state(self) -> Optional[Mapping[str, StreamState]]: """ Parent stream states are not supported for cartesian product stream slicer diff --git a/airbyte_cdk/sources/declarative/partition_routers/grouping_partition_router.py b/airbyte_cdk/sources/declarative/partition_routers/grouping_partition_router.py index a08acbbea..f4b18f5e2 100644 --- a/airbyte_cdk/sources/declarative/partition_routers/grouping_partition_router.py +++ b/airbyte_cdk/sources/declarative/partition_routers/grouping_partition_router.py @@ -140,11 +140,6 @@ def get_request_body_json( ) -> Mapping[str, Any]: return {} - def set_initial_state(self, stream_state: StreamState) -> None: - """Delegate state initialization to the underlying partition router.""" - self.underlying_partition_router.set_initial_state(stream_state) - self._state = self.underlying_partition_router.get_stream_state() - def get_stream_state(self) -> Optional[Mapping[str, StreamState]]: """Delegate state retrieval to the underlying partition router.""" return self._state diff --git a/airbyte_cdk/sources/declarative/partition_routers/list_partition_router.py b/airbyte_cdk/sources/declarative/partition_routers/list_partition_router.py index 6049cefe2..a5ed25357 100644 --- a/airbyte_cdk/sources/declarative/partition_routers/list_partition_router.py +++ b/airbyte_cdk/sources/declarative/partition_routers/list_partition_router.py @@ -108,12 +108,6 @@ def _get_request_option( else: return {} - def set_initial_state(self, stream_state: StreamState) -> None: - """ - ListPartitionRouter doesn't have parent streams - """ - pass - def get_stream_state(self) -> Optional[Mapping[str, StreamState]]: """ ListPartitionRouter doesn't have parent streams diff --git a/airbyte_cdk/sources/declarative/partition_routers/partition_router.py b/airbyte_cdk/sources/declarative/partition_routers/partition_router.py index 3a9bc3abf..a8c6ba824 100644 --- a/airbyte_cdk/sources/declarative/partition_routers/partition_router.py +++ b/airbyte_cdk/sources/declarative/partition_routers/partition_router.py @@ -15,31 +15,9 @@ class PartitionRouter(StreamSlicer): """ Base class for partition routers. Methods: - set_parent_state(stream_state): Set the state of the parent streams. - get_parent_state(): Get the state of the parent streams. + get_stream_state(): Get the state of the parent streams. """ - @abstractmethod - def set_initial_state(self, stream_state: StreamState) -> None: - """ - Set the state of the parent streams. - - This method should only be implemented if the slicer is based on some parent stream and needs to read this stream - incrementally using the state. - - Args: - stream_state (StreamState): The state of the streams to be set. The expected format is a dictionary that includes - 'parent_state' which is a dictionary of parent state names to their corresponding state. - Example: - { - "parent_state": { - "parent_stream_name_1": { ... }, - "parent_stream_name_2": { ... }, - ... - } - } - """ - @abstractmethod def get_stream_state(self) -> Optional[Mapping[str, StreamState]]: """ diff --git a/airbyte_cdk/sources/declarative/partition_routers/single_partition_router.py b/airbyte_cdk/sources/declarative/partition_routers/single_partition_router.py index 32e6a353d..e056edd2f 100644 --- a/airbyte_cdk/sources/declarative/partition_routers/single_partition_router.py +++ b/airbyte_cdk/sources/declarative/partition_routers/single_partition_router.py @@ -50,12 +50,6 @@ def get_request_body_json( def stream_slices(self) -> Iterable[StreamSlice]: yield StreamSlice(partition={}, cursor_slice={}) - def set_initial_state(self, stream_state: StreamState) -> None: - """ - SinglePartitionRouter doesn't have parent streams - """ - pass - def get_stream_state(self) -> Optional[Mapping[str, StreamState]]: """ SinglePartitionRouter doesn't have parent streams diff --git a/airbyte_cdk/sources/declarative/partition_routers/substream_partition_router.py b/airbyte_cdk/sources/declarative/partition_routers/substream_partition_router.py index 000beeff9..e41a0d9a1 100644 --- a/airbyte_cdk/sources/declarative/partition_routers/substream_partition_router.py +++ b/airbyte_cdk/sources/declarative/partition_routers/substream_partition_router.py @@ -7,24 +7,51 @@ import json import logging from dataclasses import InitVar, dataclass -from typing import TYPE_CHECKING, Any, Iterable, List, Mapping, MutableMapping, Optional, Union +from typing import ( + TYPE_CHECKING, + Any, + Iterable, + List, + Mapping, + MutableMapping, + Optional, + TypeVar, + Union, +) import dpath import requests from airbyte_cdk.models import AirbyteMessage -from airbyte_cdk.models import Type as MessageType from airbyte_cdk.sources.declarative.interpolation.interpolated_string import InterpolatedString from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter from airbyte_cdk.sources.declarative.requesters.request_option import ( RequestOption, RequestOptionType, ) -from airbyte_cdk.sources.types import Config, Record, StreamSlice, StreamState -from airbyte_cdk.utils import AirbyteTracedException +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.types import Config, StreamSlice, StreamState if TYPE_CHECKING: - from airbyte_cdk.sources.declarative.declarative_stream import DeclarativeStream + from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream + + +T = TypeVar("T") + + +def iterate_with_last_flag(generator: Iterable[T]) -> Iterable[tuple[T, bool]]: + iterator = iter(generator) + + try: + current = next(iterator) + except StopIteration: + return # Return an empty iterator + + for next_item in iterator: + yield current, False + current = next_item + + yield current, True @dataclass @@ -40,7 +67,7 @@ class ParentStreamConfig: incremental_dependency (bool): Indicates if the parent stream should be read incrementally. """ - stream: "DeclarativeStream" # Parent streams must be DeclarativeStream because we can't know which part of the stream slice is a partition for regular Stream + stream: "AbstractStream" parent_key: Union[InterpolatedString, str] partition_field: Union[InterpolatedString, str] config: Config @@ -176,59 +203,65 @@ def stream_slices(self) -> Iterable[StreamSlice]: for field_path in parent_stream_config.extra_fields ] - # read_stateless() assumes the parent is not concurrent. This is currently okay since the concurrent CDK does - # not support either substreams or RFR, but something that needs to be considered once we do - for parent_record in parent_stream.read_only_records(): - parent_partition = None - # Skip non-records (eg AirbyteLogMessage) - if isinstance(parent_record, AirbyteMessage): - self.logger.warning( - f"Parent stream {parent_stream.name} returns records of type AirbyteMessage. This SubstreamPartitionRouter is not able to checkpoint incremental parent state." - ) - if parent_record.type == MessageType.RECORD: - parent_record = parent_record.record.data # type: ignore[union-attr, assignment] # record is always a Record - else: - continue - elif isinstance(parent_record, Record): + for partition, is_last_slice in iterate_with_last_flag( + parent_stream.generate_partitions() + ): + for parent_record, is_last_record_in_slice in iterate_with_last_flag( + partition.read() + ): + # In the previous CDK implementation, state management was done internally by the stream. + # However, this could cause issues when doing availability check for example as the availability + # check would progress the state so state management was moved outside of the read method. + # Hence, we need to call the cursor here. + # Note that we call observe and close_partition before emitting the associated record as the + # ConcurrentPerPartitionCursor will associate a record with the state of the stream after the + # record was consumed. + parent_stream.cursor.observe(parent_record) parent_partition = ( parent_record.associated_slice.partition if parent_record.associated_slice else {} ) - parent_record = parent_record.data - elif not isinstance(parent_record, Mapping): - # The parent_record should only take the form of a Record, AirbyteMessage, or Mapping. Anything else is invalid - raise AirbyteTracedException( - message=f"Parent stream returned records as invalid type {type(parent_record)}" + record_data = parent_record.data + + try: + partition_value = dpath.get( + record_data, # type: ignore [arg-type] + parent_field, + ) + except KeyError: + # FIXME a log here would go a long way for debugging + continue + + # Add extra fields + extracted_extra_fields = self._extract_extra_fields( + record_data, extra_fields ) - try: - partition_value = dpath.get( - parent_record, # type: ignore [arg-type] - parent_field, + + if parent_stream_config.lazy_read_pointer: + extracted_extra_fields = { + "child_response": self._extract_child_response( + record_data, + parent_stream_config.lazy_read_pointer, # type: ignore[arg-type] # lazy_read_pointer type handeled in __post_init__ of parent_stream_config + ), + **extracted_extra_fields, + } + + if is_last_record_in_slice: + parent_stream.cursor.close_partition(partition) + if is_last_slice: + parent_stream.cursor.ensure_at_least_one_state_emitted() + + yield StreamSlice( + partition={ + partition_field: partition_value, + "parent_slice": parent_partition or {}, + }, + cursor_slice={}, + extra_fields=extracted_extra_fields, ) - except KeyError: - continue - - # Add extra fields - extracted_extra_fields = self._extract_extra_fields(parent_record, extra_fields) - - if parent_stream_config.lazy_read_pointer: - extracted_extra_fields = { - "child_response": self._extract_child_response( - parent_record, - parent_stream_config.lazy_read_pointer, # type: ignore[arg-type] # lazy_read_pointer type handeled in __post_init__ of parent_stream_config - ), - **extracted_extra_fields, - } - - yield StreamSlice( - partition={ - partition_field: partition_value, - "parent_slice": parent_partition or {}, - }, - cursor_slice={}, - extra_fields=extracted_extra_fields, - ) + + yield from [] def _extract_child_response( self, parent_record: Mapping[str, Any] | AirbyteMessage, pointer: List[InterpolatedString] @@ -278,60 +311,6 @@ def _extract_extra_fields( extracted_extra_fields[".".join(extra_field_path)] = extra_field_value return extracted_extra_fields - def set_initial_state(self, stream_state: StreamState) -> None: - """ - Set the state of the parent streams. - - If the `parent_state` key is missing from `stream_state`, migrate the child stream state to the parent stream's state format. - This migration applies only to parent streams with incremental dependencies. - - Args: - stream_state (StreamState): The state of the streams to be set. - - Example of state format: - { - "parent_state": { - "parent_stream_name1": { - "last_updated": "2023-05-27T00:00:00Z" - }, - "parent_stream_name2": { - "last_updated": "2023-05-27T00:00:00Z" - } - } - } - - Example of migrating to parent state format: - - Initial state: - { - "updated_at": "2023-05-27T00:00:00Z" - } - - After migration: - { - "updated_at": "2023-05-27T00:00:00Z", - "parent_state": { - "parent_stream_name": { - "parent_stream_cursor": "2023-05-27T00:00:00Z" - } - } - } - """ - if not stream_state: - return - - parent_state = stream_state.get("parent_state", {}) - - # Set state for each parent stream with an incremental dependency - for parent_config in self.parent_stream_configs: - if ( - not parent_state.get(parent_config.stream.name, {}) - and parent_config.incremental_dependency - ): - # Migrate child state to parent state format - parent_state = self._migrate_child_state_to_parent_state(stream_state) - - if parent_config.incremental_dependency: - parent_config.stream.state = parent_state.get(parent_config.stream.name, {}) - def _migrate_child_state_to_parent_state(self, stream_state: StreamState) -> StreamState: """ Migrate the child or global stream state into the parent stream's state format. @@ -414,7 +393,9 @@ def get_stream_state(self) -> Optional[Mapping[str, StreamState]]: parent_state = {} for parent_config in self.parent_stream_configs: if parent_config.incremental_dependency: - parent_state[parent_config.stream.name] = copy.deepcopy(parent_config.stream.state) + parent_state[parent_config.stream.name] = copy.deepcopy( + parent_config.stream.cursor.state + ) return parent_state @property diff --git a/airbyte_cdk/sources/declarative/requesters/request_options/per_partition_request_option_provider.py b/airbyte_cdk/sources/declarative/requesters/request_options/per_partition_request_option_provider.py new file mode 100644 index 000000000..423bd6f14 --- /dev/null +++ b/airbyte_cdk/sources/declarative/requesters/request_options/per_partition_request_option_provider.py @@ -0,0 +1,95 @@ +from typing import Any, Mapping, Optional, Union + +from airbyte_cdk.sources.declarative.partition_routers import PartitionRouter +from airbyte_cdk.sources.declarative.requesters.request_options import RequestOptionsProvider +from airbyte_cdk.sources.types import StreamSlice, StreamState + + +class PerPartitionRequestOptionsProvider(RequestOptionsProvider): + def __init__(self, partition_router: PartitionRouter, cursor_provider: RequestOptionsProvider): + self._partition_router = partition_router + self._cursor_provider = cursor_provider + + def get_request_params( + self, + *, + stream_state: Optional[StreamState] = None, + stream_slice: Optional[StreamSlice] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Mapping[str, Any]: + return self._partition_router.get_request_params( # type: ignore # this always returns a mapping + stream_state=stream_state, + stream_slice=StreamSlice(partition=stream_slice.partition, cursor_slice={}) + if stream_slice + else StreamSlice(partition={}, cursor_slice={}), + next_page_token=next_page_token, + ) | self._cursor_provider.get_request_params( + stream_state=stream_state, + stream_slice=StreamSlice(partition={}, cursor_slice=stream_slice.cursor_slice) + if stream_slice + else StreamSlice(partition={}, cursor_slice={}), + next_page_token=next_page_token, + ) + + def get_request_headers( + self, + *, + stream_state: Optional[StreamState] = None, + stream_slice: Optional[StreamSlice] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Mapping[str, Any]: + 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={}) + if stream_slice + else stream_slice, + next_page_token=next_page_token, + ) | self._cursor_provider.get_request_headers( + stream_state=stream_state, + stream_slice=StreamSlice(partition={}, cursor_slice=stream_slice.cursor_slice) + if stream_slice + else stream_slice, + next_page_token=next_page_token, + ) + + def get_request_body_data( + self, + *, + stream_state: Optional[StreamState] = None, + stream_slice: Optional[StreamSlice] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Union[Mapping[str, Any], str]: + return self._partition_router.get_request_body_data( # type: ignore # this always returns a mapping + stream_state=stream_state, + stream_slice=StreamSlice(partition=stream_slice.partition, cursor_slice={}) + if stream_slice + else stream_slice, + next_page_token=next_page_token, + ) | self._cursor_provider.get_request_body_data( + stream_state=stream_state, + stream_slice=StreamSlice(partition={}, cursor_slice=stream_slice.cursor_slice) + if stream_slice + else stream_slice, + next_page_token=next_page_token, + ) + + def get_request_body_json( + self, + *, + stream_state: Optional[StreamState] = None, + stream_slice: Optional[StreamSlice] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Mapping[str, Any]: + return self._partition_router.get_request_body_json( # type: ignore # this always returns a mapping + stream_state=stream_state, + stream_slice=StreamSlice(partition=stream_slice.partition, cursor_slice={}) + if stream_slice + else stream_slice, + next_page_token=next_page_token, + ) | self._cursor_provider.get_request_body_json( + stream_state=stream_state, + stream_slice=StreamSlice(partition={}, cursor_slice=stream_slice.cursor_slice) + if stream_slice + else stream_slice, + next_page_token=next_page_token, + ) diff --git a/airbyte_cdk/sources/declarative/resolvers/http_components_resolver.py b/airbyte_cdk/sources/declarative/resolvers/http_components_resolver.py index 6e85fc578..11952b963 100644 --- a/airbyte_cdk/sources/declarative/resolvers/http_components_resolver.py +++ b/airbyte_cdk/sources/declarative/resolvers/http_components_resolver.py @@ -17,6 +17,7 @@ ) from airbyte_cdk.sources.declarative.retrievers.retriever import Retriever from airbyte_cdk.sources.source import ExperimentalClassWarning +from airbyte_cdk.sources.streams.concurrent.partitions.stream_slicer import StreamSlicer from airbyte_cdk.sources.types import Config @@ -28,12 +29,14 @@ class HttpComponentsResolver(ComponentsResolver): Attributes: retriever (Retriever): The retriever used to fetch data from an API. + stream_slicer (StreamSlicer): The how the data is sliced. config (Config): Configuration object for the resolver. components_mapping (List[ComponentMappingDefinition]): List of mappings to resolve. parameters (InitVar[Mapping[str, Any]]): Additional parameters for interpolation. """ retriever: Retriever + stream_slicer: StreamSlicer config: Config components_mapping: List[ComponentMappingDefinition] parameters: InitVar[Mapping[str, Any]] @@ -88,7 +91,7 @@ def resolve_components( """ kwargs = {"stream_template_config": stream_template_config} - for stream_slice in self.retriever.stream_slices(): + for stream_slice in self.stream_slicer.stream_slices(): for components_values in self.retriever.read_records( records_schema={}, stream_slice=stream_slice ): diff --git a/airbyte_cdk/sources/declarative/retrievers/retriever.py b/airbyte_cdk/sources/declarative/retrievers/retriever.py index 3221f81e0..a4cce901c 100644 --- a/airbyte_cdk/sources/declarative/retrievers/retriever.py +++ b/airbyte_cdk/sources/declarative/retrievers/retriever.py @@ -5,6 +5,8 @@ from abc import abstractmethod from typing import Any, Iterable, Mapping, Optional +from typing_extensions import deprecated + from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.types import StreamSlice, StreamState @@ -29,11 +31,13 @@ def read_records( """ @abstractmethod + @deprecated("Stream slicing is being moved to the stream level.") def stream_slices(self) -> Iterable[Optional[StreamSlice]]: """Returns the stream slices""" @property @abstractmethod + @deprecated("State management is being moved to the stream level.") def state(self) -> StreamState: """State getter, should return state in form that can serialized to a string and send to the output as a STATE AirbyteMessage. @@ -49,5 +53,6 @@ def state(self) -> StreamState: @state.setter @abstractmethod + @deprecated("State management is being moved to the stream level.") def state(self, value: StreamState) -> None: """State setter, accept state serialized by state getter.""" diff --git a/airbyte_cdk/sources/message/repository.py b/airbyte_cdk/sources/message/repository.py index 2fc156e8c..39e7fc2ff 100644 --- a/airbyte_cdk/sources/message/repository.py +++ b/airbyte_cdk/sources/message/repository.py @@ -95,6 +95,26 @@ def consume_queue(self) -> Iterable[AirbyteMessage]: yield self._message_queue.popleft() +class StateFilteringMessageRepository(MessageRepository): + """ + This message repository is used when creating parent streams for SubstreamPartitionRouter. As the child stream + manages the state for both the child and the parents, we want to prevent parents from emitting state messages. + """ + + def __init__(self, decorated: MessageRepository) -> None: + self._decorated = decorated + + def emit_message(self, message: AirbyteMessage) -> None: + if message.type != Type.STATE: + self._decorated.emit_message(message) + + def log_message(self, level: Level, message_provider: Callable[[], LogMessage]) -> None: + self._decorated.log_message(level, message_provider) + + def consume_queue(self) -> Iterable[AirbyteMessage]: + yield from self._decorated.consume_queue() + + class LogAppenderMessageRepositoryDecorator(MessageRepository): def __init__( self, diff --git a/cdk-migrations.md b/cdk-migrations.md index d07c184d2..3445a04b8 100644 --- a/cdk-migrations.md +++ b/cdk-migrations.md @@ -1,5 +1,37 @@ # CDK Migration Guide +## Upgrading to 7.0.0 + +[Version 7.0.0](https://github.com/airbytehq/airbyte-python-cdk/releases/tag/v7.0.0) of the CDK migrates the CDK to the Concurrent CDK by removing some of the Declarative CDK concepts that are better expressed in the Concurrent CDK or that are outright incompatible with it. This changes mostly impact the Python implementations although the concept of CustomIncrementalSync has been removed from the declarative language as well. + +### CustomIncrementalSync + +Migration steps: None available + +Rationale: Our current interface for CustomIncrementalSync was assuming that the first slice would be processed before the second which would be processed before the third, etc... In a concurrent world, the multiple units of work can be done in any order. The current implementations of CustomIncrementalSync do not account for that hence are not compatible with the new version of the CDK. Also, we've rarely seen CustomIncrementalSync that were actually needed. On top of that, state management is much more complex in a concurrent world as it requires the developer to track multiple units of work and combining them to provide a simple representation of a state. For all those reason, we have decided not to support CustomIncrementalSync but if needs be, feel free to reach out to our team and we will re-evaluate the need for those. + +### CustomRetriever State + +Migration steps: Ensures that you don't implement `Retriever.state` or relying on the field `SimpleRetriever.cursor`. For more information, see the point above. + +Rationale: As mentioned above, the state has been moved outside the realm of the stream responsibilities. Therefore, it does not make sense for the retriever (which is a stream specific concept) to hold state information. This way, a connector developer wanting to implement a CustomRetriever will not have to bother about state management anymore. + +### Inheriting from Substream Partition Routing + +Migration steps: If your custom component relies on SubstreamPartitionRouter.parent_stream_configs[x].stream, make sure you migrate from the `DeclarativeStream` interface to the `AbstractStream` one. + +Rationale: `DeclarativeStream` interface is not compatible with the `AbstractStream` from the Concurrent CDK. In order to avoid maintaining two different instantiation flows (one for the SubstreamPartitionRouter and one for the Concurrent CDK), we decided to migrate `SubstreamPartitionRouter` to use `AbstractStream`. + +### CustomRetriever.stream_slices + +Migration steps: Ensures that you don't implement `Retriever.stream_slices` or relying on the field `SimpleRetriever.stream_slicer`. You can implement your own PartitionRouter to influence how stream slices are generated. + +Rationale: Generating units of work has been re-implemented as part of the Concurrent CDK as those units will be parallelized. While doing this change, there were no apparent reasons to go through the retriever in order to get the stream slices. Hence, we are deprecating this method and will remove it. + +### Possible Missing Features + +We have seen that some custom components were create just for the RequestOptionsProvider interface. There should always be an escape path for that which is the string interpolation. Given this is not enough, feel free to reach out to our team so that we can figure out a solution. + ## Upgrading to 6.34.0 [Version 6.34.0](https://github.com/airbytehq/airbyte-python-cdk/releases/tag/v6.34.0) of the CDK removes support for `stream_state` in the Jinja interpolation context. This change is breaking for any low-code connectors that use `stream_state` in the interpolation context. diff --git a/unit_tests/connector_builder/test_connector_builder_handler.py b/unit_tests/connector_builder/test_connector_builder_handler.py index 643878eec..848142df9 100644 --- a/unit_tests/connector_builder/test_connector_builder_handler.py +++ b/unit_tests/connector_builder/test_connector_builder_handler.py @@ -1141,7 +1141,9 @@ def test_read_source(mock_http_stream): for s in streams: retriever = get_retriever(s) assert isinstance(retriever, SimpleRetriever) - assert isinstance(retriever.stream_slicer, StreamSlicerTestReadDecorator) + assert isinstance( + s._stream_partition_generator._stream_slicer, StreamSlicerTestReadDecorator + ) @patch.object( @@ -1189,7 +1191,9 @@ def test_read_source_single_page_single_slice(mock_http_stream): for s in streams: retriever = get_retriever(s) assert isinstance(retriever, SimpleRetriever) - assert isinstance(retriever.stream_slicer, StreamSlicerTestReadDecorator) + assert isinstance( + s._stream_partition_generator._stream_slicer, StreamSlicerTestReadDecorator + ) @pytest.mark.parametrize( diff --git a/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py b/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py index 9cbc1ad1c..4e6bf38fb 100644 --- a/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py +++ b/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py @@ -508,47 +508,6 @@ def test_get_request_body_json( cursor.get_request_body_json(stream_slice=stream_slice) -def test_parent_state_is_set_for_per_partition_cursor( - mocked_cursor_factory, mocked_partition_router -): - # Define the parent state to be used in the test - parent_state = {"parent_cursor": "parent_state_value"} - - # Mock the partition router to return a stream slice - partition = StreamSlice( - partition={"partition_field_1": "a value", "partition_field_2": "another value"}, - cursor_slice={}, - ) - mocked_partition_router.stream_slices.return_value = [partition] - - # Mock the cursor factory to create cursors with specific states - mocked_cursor_factory.create.side_effect = [ - MockedCursorBuilder() - .with_stream_slices([{CURSOR_SLICE_FIELD: "first slice cursor value"}]) - .with_stream_state(CURSOR_STATE) - .build(), - ] - - # Mock the get_parent_state method to return the parent state - mocked_partition_router.get_stream_state.return_value = parent_state - - # Initialize the PerPartitionCursor with the mocked cursor factory and partition router - cursor = PerPartitionCursor(mocked_cursor_factory, mocked_partition_router) - - # Set the initial state, including the parent state - initial_state = { - "states": [{"partition": partition.partition, "cursor": CURSOR_STATE}], - "parent_state": parent_state, - } - cursor.set_initial_state(initial_state) - - # Verify that the parent state has been set correctly - assert cursor.get_stream_state()["parent_state"] == parent_state - - # Verify that set_parent_state was called on the partition router with the initial state - mocked_partition_router.set_initial_state.assert_called_once_with(initial_state) - - def test_get_stream_state_includes_parent_state(mocked_cursor_factory, mocked_partition_router): # Define the parent state to be used in the test parent_state = {"parent_cursor": "parent_state_value"} diff --git a/unit_tests/legacy/sources/declarative/partition_routers/__init__.py b/unit_tests/legacy/sources/declarative/partition_routers/__init__.py deleted file mode 100644 index 58b636bf9..000000000 --- a/unit_tests/legacy/sources/declarative/partition_routers/__init__.py +++ /dev/null @@ -1 +0,0 @@ -# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/legacy/sources/declarative/partition_routers/test_parent_state_stream.py b/unit_tests/legacy/sources/declarative/partition_routers/test_parent_state_stream.py deleted file mode 100644 index c3cab5500..000000000 --- a/unit_tests/legacy/sources/declarative/partition_routers/test_parent_state_stream.py +++ /dev/null @@ -1,2167 +0,0 @@ -# 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 orjson -import pytest -import requests_mock - -from airbyte_cdk.legacy.sources.declarative.manifest_declarative_source import ( - ManifestDeclarativeSource, -) -from airbyte_cdk.models import ( - AirbyteMessage, - AirbyteStateBlob, - AirbyteStateMessage, - AirbyteStateType, - AirbyteStream, - AirbyteStreamState, - ConfiguredAirbyteCatalog, - ConfiguredAirbyteStream, - DestinationSyncMode, - StreamDescriptor, - SyncMode, -) - -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_interval.get('start_date', 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"}, - ], -} - - -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]: - source = ManifestDeclarativeSource(source_config=manifest) - 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() - return list(source.read(logger, config, catalog, 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 output_data == expected_records - - # 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 - { - "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": 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-15T00:00:00Z"}, - "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 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": [], - "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 - - -@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 - { - "use_global_cursor": False, - "state": {"created_at": "2024-01-15T00: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": 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 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 - - -SUBSTREAM_MANIFEST_GLOBAL_PARENT_CURSOR: 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_interval.get('start_date', 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": { - "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", - }, - "global_substream_cursor": True, - }, - "$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"}, - ], -} -SUBSTREAM_MANIFEST_GLOBAL_PARENT_CURSOR_NO_DEPENDENCY = copy.deepcopy( - SUBSTREAM_MANIFEST_GLOBAL_PARENT_CURSOR -) -SUBSTREAM_MANIFEST_GLOBAL_PARENT_CURSOR_NO_DEPENDENCY["definitions"]["post_comment_votes_stream"][ - "retriever" -]["partition_router"]["parent_stream_configs"][0]["incremental_dependency"] = False - - -@pytest.mark.parametrize( - "test_name, manifest, mock_requests, expected_records, initial_state, expected_state", - [ - ( - "test_global_substream_cursor", - SUBSTREAM_MANIFEST_GLOBAL_PARENT_CURSOR, - [ - # 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": [ - {"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-03T00: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-03T00: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-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": [ - {"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-03T00: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", - { - "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-14T23:59:59Z", - { - "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-14T23:59:59Z", - { - "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-14T23:59:59Z", - { - "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-14T23:59:59Z", - { - "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-14T23:59:59Z", - { - "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"} - }, - } - }, - "state": {"created_at": "2024-01-04T02:03:04Z"}, - "lookback_window": 93784, - } - ), - ), - ) - ], - # Expected state - { - "state": {"created_at": "2024-01-15T00:00:00Z"}, - "lookback_window": 1, - "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"}, - }, - ], - } - }, - }, - ), - ( - "test_global_substream_cursor_no_dependency", - SUBSTREAM_MANIFEST_GLOBAL_PARENT_CURSOR_NO_DEPENDENCY, - [ - # 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-01T00:00:01Z&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-03T00: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-03T00: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-03T00: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-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-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": [ - {"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-03T00: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", - { - "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( - { - "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"} - }, - } - }, - "state": {"created_at": "2024-01-04T02:03:04Z"}, - "lookback_window": 93784, - } - ), - ), - ) - ], - # Expected state - {"state": {"created_at": "2024-01-15T00:00:00Z"}, "lookback_window": 1}, - ), - ], -) -def test_incremental_global_parent_state( - test_name, manifest, mock_requests, expected_records, initial_state, expected_state -): - run_incremental_parent_state_test( - manifest, mock_requests, expected_records, initial_state, [expected_state] - ) diff --git a/unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py b/unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py index ad6735201..d6e8c22fa 100644 --- a/unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py +++ b/unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py @@ -2195,10 +2195,11 @@ def test_only_parent_streams_use_cache(): assert not stream_1_retriever.requester.use_cache # Parent stream created for substream - assert stream_1_retriever.stream_slicer.parent_stream_configs[0].stream.name == "applications" - assert stream_1_retriever.stream_slicer.parent_stream_configs[ + stream_slicer = streams[1]._stream_partition_generator._stream_slicer + assert stream_slicer.parent_stream_configs[0].stream.name == "applications" + assert stream_slicer.parent_stream_configs[ 0 - ].stream.retriever.requester.use_cache + ].stream._stream_partition_generator._partition_factory._retriever.requester.use_cache # Main stream without caching assert streams[2].name == "jobs" diff --git a/unit_tests/sources/declarative/file/file_stream_manifest.yaml b/unit_tests/sources/declarative/file/file_stream_manifest.yaml index 256a3816c..7cba173c3 100644 --- a/unit_tests/sources/declarative/file/file_stream_manifest.yaml +++ b/unit_tests/sources/declarative/file/file_stream_manifest.yaml @@ -63,7 +63,7 @@ definitions: - "%Y-%m-%dT%H:%M:%SZ" - "%Y-%m-%dT%H:%M:%S%z" datetime_format: "%s" - cursor_field: "{{ parameters.get('cursor_field', 'updated_at') }}" + cursor_field: "updated_at" start_datetime: datetime: "{{ timestamp(config.get('start_date')) | int if config.get('start_date') else day_delta(-730, '%s') }}" start_time_option: diff --git a/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml b/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml index 1124b9ec0..b3469a070 100644 --- a/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml +++ b/unit_tests/sources/declarative/file/test_file_stream_with_filename_extractor.yaml @@ -63,7 +63,7 @@ definitions: - "%Y-%m-%dT%H:%M:%SZ" - "%Y-%m-%dT%H:%M:%S%z" datetime_format: "%s" - cursor_field: "{{ parameters.get('cursor_field', 'updated_at') }}" + cursor_field: "updated_at" start_datetime: datetime: "{{ timestamp(config.get('start_date')) | int if config.get('start_date') else day_delta(-730, '%s') }}" start_time_option: diff --git a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py index cb774bda7..3916a4da8 100644 --- a/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py +++ b/unit_tests/sources/declarative/incremental/test_concurrent_perpartitioncursor.py @@ -298,10 +298,6 @@ } 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) # Disable incremental_dependency @@ -454,6 +450,10 @@ def _run_read( ).isoformat() + "Z" PARTITION_SYNC_START_TIME = "2024-01-02T00:00:00Z" +CONFIG = { + "start_date": START_DATE, + "credentials": {"email": "email", "api_token": "api_token"}, +} @pytest.mark.parametrize( @@ -1119,6 +1119,11 @@ def run_incremental_parent_state_test( # Assert that the number of intermediate states is as expected assert len(intermediate_states) - 1 == num_intermediate_states + # Assert that ensure_at_least_one_state_emitted is called before yielding the last record from the last slice + assert ( + intermediate_states[-1][0].stream.stream_state.__dict__["parent_state"] + == intermediate_states[-2][0].stream.stream_state.__dict__["parent_state"] + ) # For each intermediate state, perform another read starting from that state for state, records_before_state in intermediate_states[:-1]: @@ -1183,6 +1188,11 @@ def run_incremental_parent_state_test( 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}]}, ), + # FIXME this is an interesting case. The previous solution would not update the parent state until `ensure_at_least_one_state_emitted` but the concurrent cursor does just before which is probably fine too + ( + f"https://api.example.com/community/posts?per_page=100&start_time={POST_1_UPDATED_AT}", + {"posts": [{"id": 1, "updated_at": POST_1_UPDATED_AT}]}, + ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", @@ -1476,6 +1486,10 @@ def run_incremental_parent_state_test( ] }, ), + ( + f"https://api.example.com/community/posts?per_page=100&start_time={POST_1_UPDATED_AT}", + {"posts": [{"id": 1, "updated_at": POST_1_UPDATED_AT}]}, + ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", @@ -1617,6 +1631,10 @@ def run_incremental_parent_state_test( ] }, ), + ( + f"https://api.example.com/community/posts?per_page=100&start_time={POST_1_UPDATED_AT}", + {"posts": [{"id": 1, "updated_at": POST_1_UPDATED_AT}]}, + ), # Fetch the first page of comments for post 1 ( "https://api.example.com/community/posts/1/comments?per_page=100", @@ -1721,7 +1739,7 @@ def run_incremental_parent_state_test( ], } }, - "lookback_window": 1, + "lookback_window": 86400, "use_global_cursor": False, "states": [ { @@ -2115,10 +2133,10 @@ def test_incremental_parent_state_migration( "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, + "cursor": {"updated_at": START_DATE}, } ], - "state": {}, + "lookback_window": 0, "use_global_cursor": False, "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, } @@ -2315,7 +2333,7 @@ def test_incremental_parent_state_no_slices( }, # Expected state { - "lookback_window": 1, + "lookback_window": 0, "use_global_cursor": False, "state": {"created_at": INITIAL_STATE_PARTITION_11_CURSOR}, "states": [ @@ -2580,16 +2598,22 @@ def test_incremental_parent_state_no_records( }, # Expected state { - # The global state, lookback window and the parent state are the same because sync failed for comment 20 + # The global state and lookback window are the same because sync failed for comment 20. + # The parent state will be updated up until the child records that were successful i.t. until post 2. + # Note that we still have an entry for the partition with post 2 but it is populated with the start date. "parent_state": { "post_comments": { "states": [ { "partition": {"id": 1, "parent_slice": {}}, - "cursor": {"updated_at": PARENT_COMMENT_CURSOR_PARTITION_1}, - } + "cursor": {"updated_at": COMMENT_10_UPDATED_AT}, + }, + { + "partition": {"id": 2, "parent_slice": {}}, + "cursor": {"updated_at": START_DATE}, + }, ], - "state": {}, + "lookback_window": 0, "use_global_cursor": False, "parent_state": {"posts": {"updated_at": PARENT_POSTS_CURSOR}}, } @@ -3904,7 +3928,7 @@ def test_given_all_partitions_finished_when_close_partition_then_final_state_emi assert len(final_state["states"]) == 2 assert final_state["state"]["updated_at"] == "2024-01-02T00:00:00Z" assert final_state["parent_state"] == {"posts": {"updated_at": "2024-01-06T00:00:00Z"}} - assert final_state["lookback_window"] == 1 + assert final_state["lookback_window"] == 86400 assert cursor._message_repository.emit_message.call_count == 2 assert mock_cursor.stream_slices.call_count == 2 # Called once for each partition diff --git a/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor_integration.py b/unit_tests/sources/declarative/incremental/test_per_partition_cursor_integration.py similarity index 67% rename from unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor_integration.py rename to unit_tests/sources/declarative/incremental/test_per_partition_cursor_integration.py index f3f8dfc72..2f2b6b2bd 100644 --- a/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor_integration.py +++ b/unit_tests/sources/declarative/incremental/test_per_partition_cursor_integration.py @@ -3,17 +3,15 @@ # import logging +from typing import Iterator, List, Tuple from unittest.mock import MagicMock, patch import orjson +import requests_mock -from airbyte_cdk.legacy.sources.declarative.incremental.per_partition_cursor import ( - PerPartitionCursor, -) -from airbyte_cdk.legacy.sources.declarative.manifest_declarative_source import ( - ManifestDeclarativeSource, -) from airbyte_cdk.models import ( + AirbyteMessage, + AirbyteRecordMessage, AirbyteStateBlob, AirbyteStateMessage, AirbyteStateType, @@ -24,9 +22,15 @@ DestinationSyncMode, StreamDescriptor, SyncMode, + Type, +) +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, ) +from airbyte_cdk.sources.declarative.incremental import ConcurrentPerPartitionCursor from airbyte_cdk.sources.declarative.retrievers.simple_retriever import SimpleRetriever from airbyte_cdk.sources.types import Record, StreamSlice +from airbyte_cdk.test.catalog_builder import CatalogBuilder, ConfiguredAirbyteStreamBuilder CURSOR_FIELD = "cursor_field" SYNC_MODE = SyncMode.incremental @@ -37,6 +41,7 @@ def __init__(self): self._incremental_sync = {} self._partition_router = {} self._substream_partition_router = {} + self._concurrency_default = None def with_list_partition_router(self, stream_name, cursor_field, partitions): self._partition_router[stream_name] = { @@ -46,7 +51,7 @@ def with_list_partition_router(self, stream_name, cursor_field, partitions): } return self - def with_substream_partition_router(self, stream_name): + def with_substream_partition_router(self, stream_name, incremental_dependency=False): self._substream_partition_router[stream_name] = { "type": "SubstreamPartitionRouter", "parent_stream_configs": [ @@ -55,6 +60,7 @@ def with_substream_partition_router(self, stream_name): "stream": "#/definitions/Rates", "parent_key": "id", "partition_field": "parent_id", + "incremental_dependency": incremental_dependency, } ], } @@ -78,9 +84,23 @@ def with_incremental_sync( "cursor_field": cursor_field, "step": step, "cursor_granularity": cursor_granularity, + "start_time_option": { + "type": "RequestOption", + "field_name": "from", + "inject_into": "request_parameter", + }, + "end_time_option": { + "type": "RequestOption", + "field_name": "to", + "inject_into": "request_parameter", + }, } return self + def with_concurrency(self, default: int) -> "ManifestBuilder": + self._concurrency_default = default + return self + def build(self): manifest = { "version": "0.34.2", @@ -104,7 +124,7 @@ def build(self): "requester": { "type": "HttpRequester", "url_base": "https://api.apilayer.com", - "path": "/exchangerates_data/latest", + "path": "/exchangerates_data/parent/{{ stream_partition['parent_id'] }}/child/latest", "http_method": "GET", }, "record_selector": { @@ -130,7 +150,7 @@ def build(self): "requester": { "type": "HttpRequester", "url_base": "https://api.apilayer.com", - "path": "/exchangerates_data/latest", + "path": "/exchangerates_data/parent/latest", "http_method": "GET", }, "record_selector": { @@ -163,11 +183,36 @@ def build(self): manifest["definitions"][stream_name]["retriever"]["partition_router"] = ( partition_router_definition ) + + if self._concurrency_default: + manifest["concurrency_level"] = { + "type": "ConcurrencyLevel", + "default_concurrency": self._concurrency_default, + } return manifest def test_given_state_for_only_some_partition_when_stream_slices_then_create_slices_using_state_or_start_from_start_datetime(): - source = ManifestDeclarativeSource( + state = [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(name="Rates", namespace=None), + stream_state=AirbyteStateBlob( + { + "states": [ + { + "partition": {"partition_field": "1"}, + "cursor": {CURSOR_FIELD: "2022-02-01"}, + } + ] + } + ), + ), + ) + ] + + source = ConcurrentDeclarativeSource( source_config=ManifestBuilder() .with_list_partition_router("Rates", "partition_field", ["1", "2"]) .with_incremental_sync( @@ -179,32 +224,45 @@ def test_given_state_for_only_some_partition_when_stream_slices_then_create_slic step="P1M", cursor_granularity="P1D", ) - .build() + .build(), + config={}, + catalog=None, + state=state, ) stream_instance = source.streams({})[0] - stream_instance.state = { - "states": [ - { - "partition": {"partition_field": "1"}, - "cursor": {CURSOR_FIELD: "2022-02-01"}, - } - ] - } - slices = stream_instance.stream_slices( - sync_mode=SYNC_MODE, - stream_state={}, - ) + partitions = stream_instance.generate_partitions() + slices = [partition.to_slice() for partition in partitions] - assert list(slices) == [ - {"partition_field": "1", "start_time": "2022-02-01", "end_time": "2022-02-28"}, - {"partition_field": "2", "start_time": "2022-01-01", "end_time": "2022-01-31"}, - {"partition_field": "2", "start_time": "2022-02-01", "end_time": "2022-02-28"}, + assert slices == [ + StreamSlice( + partition={"partition_field": "1"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ), + StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-01-01", "end_time": "2022-01-31"}, + ), + StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ), ] -def test_given_record_for_partition_when_read_then_update_state(): - source = ManifestDeclarativeSource( +def test_given_record_for_partition_when_read_then_update_state(caplog): + configured_stream = ConfiguredAirbyteStream( + stream=AirbyteStream( + name="Rates", + json_schema={}, + supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], + ), + sync_mode=SyncMode.incremental, + destination_sync_mode=DestinationSyncMode.append, + ) + catalog = ConfiguredAirbyteCatalog(streams=[configured_stream]) + + source = ConcurrentDeclarativeSource( source_config=ManifestBuilder() .with_list_partition_router("Rates", "partition_field", ["1", "2"]) .with_incremental_sync( @@ -216,45 +274,80 @@ def test_given_record_for_partition_when_read_then_update_state(): step="P1M", cursor_granularity="P1D", ) - .build() + .build(), + config={}, + catalog=catalog, + state=None, ) - stream_instance = source.streams({})[0] - list(stream_instance.stream_slices(sync_mode=SYNC_MODE)) + logger = MagicMock() - stream_slice = StreamSlice( - partition={"partition_field": "1"}, - cursor_slice={"start_time": "2022-01-01", "end_time": "2022-01-31"}, - ) - with patch.object( - SimpleRetriever, - "_read_pages", - side_effect=[ - [Record({"a record key": "a record value", CURSOR_FIELD: "2022-01-15"}, stream_slice)] - ], - ): - list( - stream_instance.read_records( - sync_mode=SYNC_MODE, - stream_slice=stream_slice, - stream_state={"states": []}, - cursor_field=CURSOR_FIELD, + stream_slice = [ + StreamSlice( + partition={"partition_field": "1"}, + cursor_slice={"start_time": "2022-01-01", "end_time": "2022-01-31"}, + ), + StreamSlice( + partition={"partition_field": "1"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ), + StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ), + StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ), + ] + + records = [ + [ + Record( + data={"a record key": "a record value", CURSOR_FIELD: "2022-01-15"}, + stream_name="Rates", + associated_slice=stream_slice[0], ) - ) + ], + [], + [], + [], + ] + + # Use caplog to capture logs + with caplog.at_level(logging.WARNING, logger="airbyte"): + with patch.object(SimpleRetriever, "_read_pages", side_effect=records): + output = list(source.read(logger, {}, catalog, None)) + + # Since the partition limit is not exceeded, we expect no warnings + logged_warnings = [record.message for record in caplog.records if record.levelname == "WARNING"] + assert len(logged_warnings) == 0 + + # Proceed with existing assertions + final_state = [ + orjson.loads(orjson.dumps(message.state.stream.stream_state)) + for message in output + if message.state + ] - assert stream_instance.state == { - "state": {}, + assert final_state[-1] == { + "lookback_window": 1, + "state": {"cursor_field": "2022-01-15"}, "use_global_cursor": False, "states": [ { "partition": {"partition_field": "1"}, "cursor": {CURSOR_FIELD: "2022-01-15"}, - } + }, + { + "partition": {"partition_field": "2"}, + "cursor": {CURSOR_FIELD: "2022-01-01"}, + }, ], } def test_substream_without_input_state(): - test_source = ManifestDeclarativeSource( + test_source = ConcurrentDeclarativeSource( source_config=ManifestBuilder() .with_substream_partition_router("AnotherStream") .with_incremental_sync( @@ -275,7 +368,10 @@ def test_substream_without_input_state(): step="P1M", cursor_granularity="P1D", ) - .build() + .build(), + config={}, + catalog=None, + state=None, ) stream_instance = test_source.streams({})[1] @@ -293,7 +389,7 @@ def test_substream_without_input_state(): [Record({"id": "2", CURSOR_FIELD: "2022-01-15"}, parent_stream_slice)], ], ): - slices = list(stream_instance.stream_slices(sync_mode=SYNC_MODE)) + slices = [partition.to_slice() for partition in stream_instance.generate_partitions()] assert list(slices) == [ StreamSlice( partition={ @@ -326,148 +422,6 @@ def test_substream_without_input_state(): ] -def test_partition_limitation(caplog): - """ - Test that when the number of partitions exceeds the maximum allowed limit in PerPartitionCursor, - the oldest partitions are dropped, and the state is updated accordingly. - - In this test, we set the maximum number of partitions to 2 and provide 3 partitions. - We verify that the state only retains information for the two most recent partitions. - """ - stream_name = "Rates" - source = ManifestDeclarativeSource( - source_config=ManifestBuilder() - .with_list_partition_router( - stream_name=stream_name, cursor_field="partition_field", partitions=["1", "2", "3"] - ) - .with_incremental_sync( - stream_name=stream_name, - start_datetime="2022-01-01", - end_datetime="2022-02-28", - datetime_format="%Y-%m-%d", - cursor_field=CURSOR_FIELD, - step="P1M", - cursor_granularity="P1D", - ) - .build() - ) - - partition_slices = [ - StreamSlice(partition={"partition_field": "1"}, cursor_slice={}), - StreamSlice(partition={"partition_field": "2"}, cursor_slice={}), - StreamSlice(partition={"partition_field": "3"}, cursor_slice={}), - ] - - records_list = [ - [ - Record( - data={"a record key": "a record value", CURSOR_FIELD: "2022-01-15"}, - associated_slice=partition_slices[0], - stream_name=stream_name, - ), - Record( - data={"a record key": "a record value", CURSOR_FIELD: "2022-01-16"}, - associated_slice=partition_slices[0], - stream_name=stream_name, - ), - ], - [ - Record( - data={"a record key": "a record value", CURSOR_FIELD: "2022-02-15"}, - associated_slice=partition_slices[0], - stream_name=stream_name, - ) - ], - [ - Record( - data={"a record key": "a record value", CURSOR_FIELD: "2022-01-16"}, - associated_slice=partition_slices[1], - stream_name=stream_name, - ) - ], - [], - [], - [ - Record( - data={"a record key": "a record value", CURSOR_FIELD: "2022-02-17"}, - associated_slice=partition_slices[2], - stream_name=stream_name, - ) - ], - ] - - configured_stream = ConfiguredAirbyteStream( - stream=AirbyteStream( - name="Rates", - json_schema={}, - supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], - ), - sync_mode=SyncMode.incremental, - destination_sync_mode=DestinationSyncMode.append, - ) - catalog = ConfiguredAirbyteCatalog(streams=[configured_stream]) - - initial_state = [ - AirbyteStateMessage( - type=AirbyteStateType.STREAM, - stream=AirbyteStreamState( - stream_descriptor=StreamDescriptor(name="post_comment_votes", namespace=None), - stream_state=AirbyteStateBlob( - { - "states": [ - { - "partition": {"partition_field": "1"}, - "cursor": {CURSOR_FIELD: "2022-01-01"}, - }, - { - "partition": {"partition_field": "2"}, - "cursor": {CURSOR_FIELD: "2022-01-02"}, - }, - { - "partition": {"partition_field": "3"}, - "cursor": {CURSOR_FIELD: "2022-01-03"}, - }, - ] - } - ), - ), - ) - ] - logger = MagicMock() - - # Use caplog to capture logs - with caplog.at_level(logging.WARNING, logger="airbyte"): - with patch.object(SimpleRetriever, "_read_pages", side_effect=records_list): - with patch.object(PerPartitionCursor, "DEFAULT_MAX_PARTITIONS_NUMBER", 2): - output = list(source.read(logger, {}, catalog, initial_state)) - - # Check if the warning was logged - logged_messages = [record.message for record in caplog.records if record.levelname == "WARNING"] - warning_message = 'The maximum number of partitions has been reached. Dropping the oldest partition: {"partition_field":"1"}. Over limit: 1.' - assert warning_message in logged_messages - - final_state = [ - orjson.loads(orjson.dumps(message.state.stream.stream_state)) - for message in output - if message.state - ] - assert final_state[-1] == { - "lookback_window": 1, - "state": {"cursor_field": "2022-02-17"}, - "use_global_cursor": False, - "states": [ - { - "partition": {"partition_field": "2"}, - "cursor": {CURSOR_FIELD: "2022-01-16"}, - }, - { - "partition": {"partition_field": "3"}, - "cursor": {CURSOR_FIELD: "2022-02-17"}, - }, - ], - } - - def test_perpartition_with_fallback(caplog): """ Test that when the number of partitions exceeds the limit in PerPartitionCursor, @@ -476,20 +430,6 @@ def test_perpartition_with_fallback(caplog): This test also checks that the appropriate warning logs are emitted when the partition limit is exceeded. """ stream_name = "Rates" - source = ManifestDeclarativeSource( - source_config=ManifestBuilder() - .with_list_partition_router("Rates", "partition_field", ["1", "2", "3", "4", "5", "6"]) - .with_incremental_sync( - stream_name=stream_name, - start_datetime="2022-01-01", - end_datetime="2022-02-28", - datetime_format="%Y-%m-%d", - cursor_field=CURSOR_FIELD, - step="P1M", - cursor_granularity="P1D", - ) - .build() - ) partition_slices = [ StreamSlice(partition={"partition_field": str(i)}, cursor_slice={}) for i in range(1, 7) @@ -608,23 +548,28 @@ def test_perpartition_with_fallback(caplog): ] logger = MagicMock() - # Use caplog to capture logs - with caplog.at_level(logging.WARNING, logger="airbyte"): - with patch.object(SimpleRetriever, "_read_pages", side_effect=records_list): - with patch.object(PerPartitionCursor, "DEFAULT_MAX_PARTITIONS_NUMBER", 2): - output = list(source.read(logger, {}, catalog, initial_state)) - - # Check if the warnings were logged - expected_warning_messages = [ - 'The maximum number of partitions has been reached. Dropping the oldest partition: {"partition_field":"1"}. Over limit: 1.', - 'The maximum number of partitions has been reached. Dropping the oldest partition: {"partition_field":"2"}. Over limit: 2.', - 'The maximum number of partitions has been reached. Dropping the oldest partition: {"partition_field":"3"}. Over limit: 3.', - ] - - logged_messages = [record.message for record in caplog.records if record.levelname == "WARNING"] + source = ConcurrentDeclarativeSource( + source_config=ManifestBuilder() + .with_list_partition_router("Rates", "partition_field", ["1", "2", "3", "4", "5", "6"]) + .with_incremental_sync( + stream_name=stream_name, + start_datetime="2022-01-01", + end_datetime="2022-02-28", + datetime_format="%Y-%m-%d", + cursor_field=CURSOR_FIELD, + step="P1M", + cursor_granularity="P1D", + ) + .build(), + config={}, + catalog=catalog, + state=initial_state, + ) - for expected_message in expected_warning_messages: - assert expected_message in logged_messages + with patch.object(SimpleRetriever, "_read_pages", side_effect=records_list): + with patch.object(ConcurrentPerPartitionCursor, "DEFAULT_MAX_PARTITIONS_NUMBER", 2): + with patch.object(ConcurrentPerPartitionCursor, "SWITCH_TO_GLOBAL_LIMIT", 1): + output = list(source.read(logger, {}, catalog, initial_state)) # Proceed with existing assertions final_state = [ @@ -646,69 +591,108 @@ def test_per_partition_cursor_within_limit(caplog): This test also checks that no warning logs are emitted when the partition limit is not exceeded. """ - source = ManifestDeclarativeSource( - source_config=ManifestBuilder() - .with_list_partition_router("Rates", "partition_field", ["1", "2", "3"]) - .with_incremental_sync( - "Rates", - start_datetime="2022-01-01", - end_datetime="2022-03-31", - datetime_format="%Y-%m-%d", - cursor_field=CURSOR_FIELD, - step="P1M", - cursor_granularity="P1D", - ) - .build() - ) - partition_slices = [ StreamSlice(partition={"partition_field": str(i)}, cursor_slice={}) for i in range(1, 4) ] + slice_range_2022_01_01_partition_1 = StreamSlice( + partition={"partition_field": "1"}, + cursor_slice={"start_time": "2022-01-01", "end_time": "2022-01-31"}, + ) + slice_range_2022_02_01_partition_1 = StreamSlice( + partition={"partition_field": "1"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ) + slice_range_2022_03_01_partition_1 = StreamSlice( + partition={"partition_field": "1"}, + cursor_slice={"start_time": "2022-03-01", "end_time": "2022-03-31"}, + ) + slice_range_2022_01_01_partition_2 = StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-01-01", "end_time": "2022-01-31"}, + ) + slice_range_2022_02_01_partition_2 = StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ) + slice_range_2022_03_01_partition_2 = StreamSlice( + partition={"partition_field": "2"}, + cursor_slice={"start_time": "2022-03-01", "end_time": "2022-03-31"}, + ) + slice_range_2022_01_01_partition_3 = StreamSlice( + partition={"partition_field": "3"}, + cursor_slice={"start_time": "2022-01-01", "end_time": "2022-01-31"}, + ) + slice_range_2022_02_01_partition_3 = StreamSlice( + partition={"partition_field": "3"}, + cursor_slice={"start_time": "2022-02-01", "end_time": "2022-02-28"}, + ) + slice_range_2022_03_01_partition_3 = StreamSlice( + partition={"partition_field": "3"}, + cursor_slice={"start_time": "2022-03-01", "end_time": "2022-03-31"}, + ) records_list = [ [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-01-15"}, partition_slices[0] + {"a record key": "a record value", CURSOR_FIELD: "2022-01-15"}, + stream_name="Rates", + associated_slice=slice_range_2022_01_01_partition_1, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-02-20"}, partition_slices[0] + {"a record key": "a record value", CURSOR_FIELD: "2022-02-20"}, + stream_name="Rates", + associated_slice=slice_range_2022_02_01_partition_1, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-03-25"}, partition_slices[0] + {"a record key": "a record value", CURSOR_FIELD: "2022-03-25"}, + stream_name="Rates", + associated_slice=slice_range_2022_03_01_partition_1, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-01-16"}, partition_slices[1] + {"a record key": "a record value", CURSOR_FIELD: "2022-01-16"}, + stream_name="Rates", + associated_slice=slice_range_2022_01_01_partition_2, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-02-18"}, partition_slices[1] + {"a record key": "a record value", CURSOR_FIELD: "2022-02-18"}, + stream_name="Rates", + associated_slice=slice_range_2022_02_01_partition_2, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-03-28"}, partition_slices[1] + {"a record key": "a record value", CURSOR_FIELD: "2022-03-28"}, + stream_name="Rates", + associated_slice=slice_range_2022_03_01_partition_2, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-01-17"}, partition_slices[2] + {"a record key": "a record value", CURSOR_FIELD: "2022-01-17"}, + stream_name="Rates", + associated_slice=slice_range_2022_01_01_partition_3, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-02-19"}, partition_slices[2] + {"a record key": "a record value", CURSOR_FIELD: "2022-02-19"}, + stream_name="Rates", + associated_slice=slice_range_2022_02_01_partition_3, ) ], [ Record( - {"a record key": "a record value", CURSOR_FIELD: "2022-03-29"}, partition_slices[2] + {"a record key": "a record value", CURSOR_FIELD: "2022-03-29"}, + stream_name="Rates", + associated_slice=slice_range_2022_03_01_partition_3, ) ], ] @@ -727,10 +711,28 @@ def test_per_partition_cursor_within_limit(caplog): initial_state = {} logger = MagicMock() + source = ConcurrentDeclarativeSource( + source_config=ManifestBuilder() + .with_list_partition_router("Rates", "partition_field", ["1", "2", "3"]) + .with_incremental_sync( + "Rates", + start_datetime="2022-01-01", + end_datetime="2022-03-31", + datetime_format="%Y-%m-%d", + cursor_field=CURSOR_FIELD, + step="P1M", + cursor_granularity="P1D", + ) + .build(), + config={}, + catalog=catalog, + state=initial_state, + ) + # Use caplog to capture logs with caplog.at_level(logging.WARNING, logger="airbyte"): with patch.object(SimpleRetriever, "_read_pages", side_effect=records_list): - with patch.object(PerPartitionCursor, "DEFAULT_MAX_PARTITIONS_NUMBER", 5): + with patch.object(ConcurrentPerPartitionCursor, "DEFAULT_MAX_PARTITIONS_NUMBER", 5): output = list(source.read(logger, {}, catalog, initial_state)) # Since the partition limit is not exceeded, we expect no warnings @@ -762,3 +764,87 @@ def test_per_partition_cursor_within_limit(caplog): }, ], } + + +def test_parent_stream_is_updated_after_parent_record_fully_consumed(): + source = ConcurrentDeclarativeSource( + source_config=ManifestBuilder() + .with_substream_partition_router("AnotherStream", incremental_dependency=True) + .with_incremental_sync( + "AnotherStream", + start_datetime="2022-01-01", + end_datetime="2022-02-28", + datetime_format="%Y-%m-%d", + cursor_field=CURSOR_FIELD, + step="P1Y", + cursor_granularity="P1D", + ) + .with_incremental_sync( + "Rates", + start_datetime="2022-01-01", + end_datetime="2022-02-28", + datetime_format="%Y-%m-%d", + cursor_field=CURSOR_FIELD, + step="P1Y", + cursor_granularity="P1D", + ) + .with_concurrency(1) # so that we know partition 1 gets processed before 2 + .build(), + config={}, + catalog=None, + state=None, + ) + + with requests_mock.Mocker() as m: + # Request for parent stream + m.get( + "https://api.apilayer.com/exchangerates_data/parent/latest?from=2022-01-01&to=2022-02-28", + json=[{"id": "1", CURSOR_FIELD: "2022-02-01"}, {"id": "2", CURSOR_FIELD: "2022-02-10"}], + ) + + # Requests for child stream + record_from_first_partition = {"id": "child_1.1"} + m.get( + "https://api.apilayer.com/exchangerates_data/parent/1/child/latest?from=2022-01-01&to=2022-02-28", + json=[record_from_first_partition], + ) + + record_from_second_partition = {"id": "child_1.2"} + m.get( + "https://api.apilayer.com/exchangerates_data/parent/2/child/latest?from=2022-01-01&to=2022-02-28", + json=[record_from_second_partition], + ) + + message_iterator = source.read( + MagicMock(), + {}, + CatalogBuilder() + .with_stream(ConfiguredAirbyteStreamBuilder().with_name("AnotherStream")) + .build(), + None, + ) + + records, state = get_records_until_state_message(message_iterator) + assert len(records) == 1 and records[0].data == record_from_first_partition + assert state.stream.stream_state.__dict__["parent_state"] == { + "Rates": {"cursor_field": "2022-01-01"} + } # state cursor value == start_datetime + + records, state = get_records_until_state_message(message_iterator) + assert len(records) == 1 and records[0].data == record_from_second_partition + assert state.stream.stream_state.__dict__["parent_state"] == { + "Rates": {"cursor_field": "2022-02-10"} + } # state cursor value == most_recent_cursor_value + + +def get_records_until_state_message( + message_iterator: Iterator[AirbyteMessage], +) -> Tuple[List[AirbyteRecordMessage], AirbyteStateMessage]: + records = [] + for message in message_iterator: + if message.type == Type.RECORD: + records.append(message.record) + elif message.type == Type.STATE: + return records, message.state + + raise ValueError("No state message encountered") 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 b543354f7..3b2eaf03b 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 @@ -163,6 +163,7 @@ from airbyte_cdk.sources.declarative.transformations import AddFields, RemoveFields from airbyte_cdk.sources.declarative.transformations.add_fields import AddedFieldDefinition from airbyte_cdk.sources.declarative.yaml_declarative_source import YamlDeclarativeSource +from airbyte_cdk.sources.message.repository import StateFilteringMessageRepository from airbyte_cdk.sources.streams.call_rate import MovingWindowCallRatePolicy from airbyte_cdk.sources.streams.concurrent.clamping import ( ClampingEndProvider, @@ -198,6 +199,8 @@ "start_time": "2024-01-01T00:00:00.000+00:00", "end_time": "2025-01-01T00:00:00.000+00:00", } +CONFIG_START_TIME = ab_datetime_parse(input_config["start_time"]) +CONFIG_END_TIME = ab_datetime_parse(input_config["end_time"]) def get_factory_with_parameters( @@ -715,13 +718,14 @@ def test_create_substream_partition_router(): model_type=SubstreamPartitionRouterModel, component_definition=partition_router_manifest, config=input_config, + stream_name="child_stream", ) assert isinstance(partition_router, SubstreamPartitionRouter) parent_stream_configs = partition_router.parent_stream_configs assert len(parent_stream_configs) == 2 - assert isinstance(parent_stream_configs[0].stream, DeclarativeStream) - assert isinstance(parent_stream_configs[1].stream, DeclarativeStream) + assert isinstance(parent_stream_configs[0].stream, DefaultStream) + assert isinstance(parent_stream_configs[1].stream, DefaultStream) assert partition_router.parent_stream_configs[0].parent_key.eval({}) == "id" assert partition_router.parent_stream_configs[0].partition_field.eval({}) == "repository_id" @@ -919,27 +923,80 @@ def test_stream_with_incremental_and_retriever_with_partition_router(): model_type=DeclarativeStreamModel, component_definition=stream_manifest, config=input_config ) - assert isinstance(stream, DeclarativeStream) - assert isinstance(stream.retriever, SimpleRetriever) - assert isinstance(stream.retriever.stream_slicer, PerPartitionWithGlobalCursor) + assert isinstance(stream, DefaultStream) + retriever = get_retriever(stream) + assert isinstance(retriever, SimpleRetriever) + assert isinstance(stream.cursor, ConcurrentPerPartitionCursor) - datetime_stream_slicer = ( - stream.retriever.stream_slicer._per_partition_cursor._cursor_factory.create() + concurrent_cursor = ( + stream.cursor._cursor_factory.create( + {}, timedelta(0) + ) # FIXME should we be allowed to pass `None` instead of `{}` ) - assert isinstance(datetime_stream_slicer, DatetimeBasedCursor) - assert isinstance(datetime_stream_slicer._start_datetime, MinMaxDatetime) - assert datetime_stream_slicer._start_datetime.datetime.string == "{{ config['start_time'] }}" - assert isinstance(datetime_stream_slicer._end_datetime, MinMaxDatetime) - assert datetime_stream_slicer._end_datetime.datetime.string == "{{ config['end_time'] }}" - assert datetime_stream_slicer.step == "P10D" - assert datetime_stream_slicer.cursor_field.string == "created" + assert isinstance(concurrent_cursor, ConcurrentCursor) + assert concurrent_cursor._start == CONFIG_START_TIME + assert concurrent_cursor._end_provider() == CONFIG_END_TIME + assert concurrent_cursor._slice_range == timedelta(days=10) + assert concurrent_cursor.cursor_field.cursor_field_key == "created" - list_stream_slicer = stream.retriever.stream_slicer._partition_router + list_stream_slicer = stream.cursor._partition_router assert isinstance(list_stream_slicer, ListPartitionRouter) assert list_stream_slicer.values == ["airbyte", "airbyte-cloud"] assert list_stream_slicer._cursor_field.string == "a_key" +def test_stream_with_custom_retriever_and_transformations(): + content = """ +a_stream: + type: DeclarativeStream + primary_key: id + schema_loader: + type: InlineSchemaLoader + schema: + $schema: "http://json-schema.org/draft-07/schema" + type: object + properties: + id: + type: string + retriever: + type: CustomRetriever + class_name: unit_tests.sources.declarative.parsers.testing_components.TestingCustomRetriever + name: "{{ parameters['name'] }}" + decoder: + type: JsonDecoder + requester: + type: HttpRequester + name: "{{ parameters['name'] }}" + url_base: "https://api.sendgrid.com/v3/" + http_method: "GET" + record_selector: + type: RecordSelector + extractor: + type: DpathExtractor + field_path: ["records"] + transformations: + - type: AddFields + fields: + - path: ["extra"] + value: "{{ response.to_add }}" + $parameters: + name: a_stream +""" + + parsed_manifest = YamlDeclarativeSource._parse(content) + resolved_manifest = resolver.preprocess_manifest(parsed_manifest) + stream_manifest = transformer.propagate_types_and_parameters( + "", resolved_manifest["a_stream"], {} + ) + + stream = factory.create_component( + model_type=DeclarativeStreamModel, component_definition=stream_manifest, config=input_config + ) + + assert isinstance(stream, DefaultStream) + assert get_retriever(stream).record_selector.transformations + + @pytest.mark.parametrize( "use_legacy_state", [ @@ -1011,9 +1068,10 @@ def test_stream_with_incremental_and_async_retriever_with_partition_router(use_l config=connector_config, ) - assert isinstance(stream, DeclarativeStream) - assert isinstance(stream.retriever, AsyncRetriever) - stream_slicer = stream.retriever.stream_slicer.stream_slicer + assert isinstance(stream, DefaultStream) + retriever = get_retriever(stream) + assert isinstance(retriever, AsyncRetriever) + stream_slicer = retriever.stream_slicer.stream_slicer assert isinstance(stream_slicer, ConcurrentPerPartitionCursor) assert stream_slicer.state == stream_state import json @@ -1273,12 +1331,13 @@ def test_client_side_incremental_with_partition_router(): model_type=DeclarativeStreamModel, component_definition=stream_manifest, config=input_config ) + retriever = get_retriever(stream) assert isinstance( - stream.retriever.record_selector.record_filter, ClientSideIncrementalRecordFilterDecorator + retriever.record_selector.record_filter, ClientSideIncrementalRecordFilterDecorator ) - assert stream.retriever.record_selector.transform_before_filtering == True + assert retriever.record_selector.transform_before_filtering == True assert isinstance( - stream.retriever.record_selector.record_filter._cursor, + retriever.record_selector.record_filter._cursor, ConcurrentPerPartitionCursor, ) @@ -2047,11 +2106,20 @@ def test_custom_components_do_not_contain_extra_fields(): } custom_substream_partition_router = factory.create_component( - CustomPartitionRouterModel, custom_substream_partition_router_manifest, input_config + CustomPartitionRouterModel, + custom_substream_partition_router_manifest, + input_config, + stream_name="child_stream_name", ) assert isinstance(custom_substream_partition_router, TestingCustomSubstreamPartitionRouter) assert len(custom_substream_partition_router.parent_stream_configs) == 1 + assert isinstance( + custom_substream_partition_router.parent_stream_configs[ + 0 + ].stream.cursor._message_repository, + StateFilteringMessageRepository, + ) assert custom_substream_partition_router.parent_stream_configs[0].parent_key.eval({}) == "id" assert ( custom_substream_partition_router.parent_stream_configs[0].partition_field.eval({}) @@ -2114,7 +2182,10 @@ def test_parse_custom_component_fields_if_subcomponent(): } custom_substream_partition_router = factory.create_component( - CustomPartitionRouterModel, custom_substream_partition_router_manifest, input_config + CustomPartitionRouterModel, + custom_substream_partition_router_manifest, + input_config, + stream_name="child_stream_name", ) assert isinstance(custom_substream_partition_router, TestingCustomSubstreamPartitionRouter) assert custom_substream_partition_router.custom_field == "here" @@ -2462,8 +2533,8 @@ def test_default_schema_loader(self): "values": "{{config['repos']}}", "cursor_field": "a_key", }, - PerPartitionWithGlobalCursor, - DeclarativeStream, + ConcurrentPerPartitionCursor, + DefaultStream, id="test_create_simple_retriever_with_incremental_and_partition_router", ), pytest.param( @@ -2488,8 +2559,8 @@ def test_default_schema_loader(self): "cursor_field": "b_key", }, ], - PerPartitionWithGlobalCursor, - DeclarativeStream, + ConcurrentPerPartitionCursor, + DefaultStream, id="test_create_simple_retriever_with_partition_routers_multiple_components", ), pytest.param( @@ -2545,14 +2616,10 @@ def test_merge_incremental_and_partition_router( assert isinstance(stream_slicer, expected_router_type) if incremental and partition_router: - assert isinstance(retriever.stream_slicer, PerPartitionWithGlobalCursor) + assert isinstance(stream.cursor, ConcurrentPerPartitionCursor) if isinstance(partition_router, list) and len(partition_router) > 1: - assert isinstance( - retriever.stream_slicer._partition_router, CartesianProductStreamSlicer - ) - assert len(retriever.stream_slicer._partition_router.stream_slicers) == len( - partition_router - ) + assert isinstance(stream.cursor._partition_router, CartesianProductStreamSlicer) + assert len(stream.cursor._partition_router.stream_slicers) == len(partition_router) def test_simple_retriever_emit_log_messages(): @@ -2597,7 +2664,6 @@ def test_simple_retriever_emit_log_messages(): assert retriever.log_formatter(response) == connector_builder_factory._get_log_formatter( None, retriever.name )(response) - assert isinstance(retriever.stream_slicer, StreamSlicerTestReadDecorator) def test_create_page_increment(): @@ -2948,10 +3014,6 @@ def test_use_request_options_provider_for_datetime_based_cursor(): assert retriever.primary_key == "id" assert retriever.name == "Test" - assert isinstance(retriever.cursor, DatetimeBasedCursor) - assert isinstance(retriever.stream_slicer, StreamSlicerTestReadDecorator) - assert isinstance(retriever.stream_slicer.wrapped_slicer, DatetimeBasedCursor) - assert isinstance(retriever.request_option_provider, DatetimeBasedRequestOptionsProvider) assert ( retriever.request_option_provider.start_time_option.inject_into @@ -2973,79 +3035,6 @@ def test_use_request_options_provider_for_datetime_based_cursor(): assert retriever.request_option_provider._partition_field_end.string == "end_time" -def test_do_not_separate_request_options_provider_for_non_datetime_based_cursor(): - # This test validates that we're only using the dedicated RequestOptionsProvider for DatetimeBasedCursor and using the - # existing StreamSlicer for other types of cursors and partition routing. Once everything is migrated this test can be deleted - - config = { - "start_time": "2024-01-01T00:00:00.000000+0000", - } - - simple_retriever_model = { - "type": "SimpleRetriever", - "record_selector": { - "type": "RecordSelector", - "extractor": { - "type": "DpathExtractor", - "field_path": [], - }, - }, - "requester": { - "type": "HttpRequester", - "name": "list", - "url_base": "orange.com", - "path": "/v1/api", - }, - } - - datetime_based_cursor = DatetimeBasedCursor( - start_datetime=MinMaxDatetime(datetime="{{ config.start_time }}", parameters={}), - step="P5D", - cursor_field="updated_at", - datetime_format="%Y-%m-%dT%H:%M:%S.%f%z", - cursor_granularity="PT1S", - is_compare_strictly=True, - config=config, - parameters={}, - ) - - list_partition_router = ListPartitionRouter( - cursor_field="id", - values=["four", "oh", "eight"], - config=config, - parameters={}, - ) - - per_partition_cursor = PerPartitionCursor( - cursor_factory=CursorFactory(lambda: datetime_based_cursor), - partition_router=list_partition_router, - ) - - connector_builder_factory = ModelToComponentFactory(emit_connector_builder_messages=True) - retriever = connector_builder_factory.create_component( - model_type=SimpleRetrieverModel, - component_definition=simple_retriever_model, - config={}, - name="Test", - primary_key="id", - stream_slicer=per_partition_cursor, - request_options_provider=None, - transformations=[], - ) - - assert isinstance(retriever, SimpleRetriever) - assert retriever.primary_key == "id" - assert retriever.name == "Test" - - assert isinstance(retriever.cursor, PerPartitionCursor) - assert isinstance(retriever.stream_slicer, StreamSlicerTestReadDecorator) - assert isinstance(retriever.stream_slicer.wrapped_slicer, PerPartitionCursor) - - assert isinstance(retriever.request_option_provider, PerPartitionCursor) - assert isinstance(retriever.request_option_provider._cursor_factory, CursorFactory) - assert retriever.request_option_provider._partition_router == list_partition_router - - def test_use_default_request_options_provider(): simple_retriever_model = { "type": "SimpleRetriever", @@ -3080,8 +3069,6 @@ def test_use_default_request_options_provider(): assert retriever.primary_key == "id" assert retriever.name == "Test" - assert isinstance(retriever.stream_slicer, StreamSlicerTestReadDecorator) - assert isinstance(retriever.stream_slicer.wrapped_slicer, SinglePartitionRouter) assert isinstance(retriever.request_option_provider, DefaultRequestOptionsProvider) @@ -3946,6 +3933,7 @@ def test_create_grouping_partition_router_with_underlying_router(): model_type=GroupingPartitionRouterModel, component_definition=partition_router_manifest, config=input_config, + stream_name="child_stream", ) # Test the created partition router @@ -3956,7 +3944,7 @@ def test_create_grouping_partition_router_with_underlying_router(): # Test the underlying partition router parent_stream_configs = partition_router.underlying_partition_router.parent_stream_configs assert len(parent_stream_configs) == 1 - assert isinstance(parent_stream_configs[0].stream, DeclarativeStream) + assert isinstance(parent_stream_configs[0].stream, DefaultStream) assert parent_stream_configs[0].parent_key.eval({}) == "id" assert parent_stream_configs[0].partition_field.eval({}) == "repository_id" @@ -4004,6 +3992,7 @@ def test_create_grouping_partition_router_invalid_group_size(): model_type=GroupingPartitionRouterModel, component_definition=partition_router_manifest, config=input_config, + stream_name="child_stream", ) @@ -4055,6 +4044,7 @@ def test_create_grouping_partition_router_substream_with_request_option(): model_type=GroupingPartitionRouterModel, component_definition=partition_router_manifest, config=input_config, + stream_name="child_stream", ) diff --git a/unit_tests/sources/declarative/parsers/testing_components.py b/unit_tests/sources/declarative/parsers/testing_components.py index ab9ae5346..0b9a68e6b 100644 --- a/unit_tests/sources/declarative/parsers/testing_components.py +++ b/unit_tests/sources/declarative/parsers/testing_components.py @@ -13,6 +13,7 @@ DefaultPaginator, PaginationStrategy, ) +from airbyte_cdk.sources.declarative.retrievers import SimpleRetriever @dataclass @@ -43,3 +44,8 @@ class TestingCustomSubstreamPartitionRouter(SubstreamPartitionRouter): custom_field: str custom_pagination_strategy: PaginationStrategy + + +@dataclass +class TestingCustomRetriever(SimpleRetriever): + pass diff --git a/unit_tests/sources/declarative/partition_routers/helpers.py b/unit_tests/sources/declarative/partition_routers/helpers.py deleted file mode 100644 index b64b40d5e..000000000 --- a/unit_tests/sources/declarative/partition_routers/helpers.py +++ /dev/null @@ -1,93 +0,0 @@ -# -# Copyright (c) 2025 Airbyte, Inc., all rights reserved. -# - -from typing import Any, Iterable, List, Mapping, Optional, Union - -from airbyte_cdk.models import SyncMode -from airbyte_cdk.sources.declarative.declarative_stream import DeclarativeStream -from airbyte_cdk.sources.declarative.interpolation import InterpolatedString -from airbyte_cdk.sources.streams.checkpoint import Cursor -from airbyte_cdk.sources.types import Record, StreamSlice - - -class MockStream(DeclarativeStream): - def __init__(self, slices, records, name, cursor_field="", cursor=None): - self.config = {} - self._slices = slices - self._records = records - self._stream_cursor_field = ( - InterpolatedString.create(cursor_field, parameters={}) - if isinstance(cursor_field, str) - else cursor_field - ) - self._name = name - self._state = {"states": []} - self._cursor = cursor - - @property - def name(self) -> str: - return self._name - - @property - def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: - return "id" - - @property - def state(self) -> Mapping[str, Any]: - return self._state - - @state.setter - def state(self, value: Mapping[str, Any]) -> None: - self._state = value - - @property - def is_resumable(self) -> bool: - return bool(self._cursor) - - def get_cursor(self) -> Optional[Cursor]: - return self._cursor - - def stream_slices( - self, - *, - sync_mode: SyncMode, - cursor_field: List[str] = None, - stream_state: Mapping[str, Any] = None, - ) -> Iterable[Optional[StreamSlice]]: - for s in self._slices: - if isinstance(s, StreamSlice): - yield s - else: - yield StreamSlice(partition=s, cursor_slice={}) - - def read_records( - self, - sync_mode: SyncMode, - cursor_field: List[str] = None, - stream_slice: Mapping[str, Any] = None, - stream_state: Mapping[str, Any] = None, - ) -> Iterable[Mapping[str, Any]]: - # The parent stream's records should always be read as full refresh - assert sync_mode == SyncMode.full_refresh - - if not stream_slice: - result = self._records - else: - result = [ - Record(data=r, associated_slice=stream_slice, stream_name=self.name) - for r in self._records - if r["slice"] == stream_slice["slice"] - ] - - yield from result - - # Update the state only after reading the full slice - cursor_field = self._stream_cursor_field.eval(config=self.config) - if stream_slice and cursor_field and result: - self._state["states"].append( - {cursor_field: result[-1][cursor_field], "partition": stream_slice["slice"]} - ) - - def get_json_schema(self) -> Mapping[str, Any]: - return {} diff --git a/unit_tests/sources/declarative/partition_routers/test_cartesian_product_partition_router.py b/unit_tests/sources/declarative/partition_routers/test_cartesian_product_partition_router.py index 65122adc9..232a77e12 100644 --- a/unit_tests/sources/declarative/partition_routers/test_cartesian_product_partition_router.py +++ b/unit_tests/sources/declarative/partition_routers/test_cartesian_product_partition_router.py @@ -1,6 +1,7 @@ # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # +from unittest.mock import Mock import pytest as pytest @@ -10,17 +11,13 @@ from airbyte_cdk.sources.declarative.partition_routers import ( CartesianProductStreamSlicer, ListPartitionRouter, -) -from airbyte_cdk.sources.declarative.partition_routers.substream_partition_router import ( - ParentStreamConfig, - SubstreamPartitionRouter, + PartitionRouter, ) from airbyte_cdk.sources.declarative.requesters.request_option import ( RequestOption, RequestOptionType, ) from airbyte_cdk.sources.types import StreamSlice -from unit_tests.sources.declarative.partition_routers.helpers import MockStream @pytest.mark.parametrize( @@ -176,66 +173,26 @@ def test_substream_slicer(test_name, stream_slicers, expected_slices): assert slices == expected_slices -@pytest.mark.parametrize( - "test_name, stream_slicers, expected_slices", - [ - ( - "test_single_stream_slicer", - [ - SubstreamPartitionRouter( - parent_stream_configs=[ - ParentStreamConfig( - stream=MockStream( - [{}], - [ - {"a": {"b": 0}, "extra_field_key": "extra_field_value_0"}, - {"a": {"b": 1}, "extra_field_key": "extra_field_value_1"}, - {"a": {"c": 2}, "extra_field_key": "extra_field_value_2"}, - {"a": {"b": 3}, "extra_field_key": "extra_field_value_3"}, - ], - "first_stream", - ), - parent_key="a/b", - partition_field="first_stream_id", - parameters={}, - config={}, - extra_fields=[["extra_field_key"]], - ) - ], - parameters={}, - config={}, - ), - ], - [ - StreamSlice( - partition={"first_stream_id": 0, "parent_slice": {}}, - cursor_slice={}, - extra_fields={"extra_field_key": "extra_field_value_0"}, - ), - StreamSlice( - partition={"first_stream_id": 1, "parent_slice": {}}, - cursor_slice={}, - extra_fields={"extra_field_key": "extra_field_value_1"}, - ), - StreamSlice( - partition={"first_stream_id": 3, "parent_slice": {}}, - cursor_slice={}, - extra_fields={"extra_field_key": "extra_field_value_3"}, - ), - ], - ) - ], -) -def test_substream_slicer_with_extra_fields(test_name, stream_slicers, expected_slices): - slicer = CartesianProductStreamSlicer(stream_slicers=stream_slicers, parameters={}) - slices = [s for s in slicer.stream_slices()] - partitions = [s.partition for s in slices] - expected_partitions = [s.partition for s in expected_slices] - assert partitions == expected_partitions - - extra_fields = [s.extra_fields for s in slices] - expected_extra_fields = [s.extra_fields for s in expected_slices] - assert extra_fields == expected_extra_fields +def test_substream_slicer_with_extra_fields(): + decorated_slicer = Mock(spec=PartitionRouter) + decorated_slicer.stream_slices.return_value = iter( + [ + StreamSlice( + partition={"first_stream_id": 0, "parent_slice": {}}, + cursor_slice={}, + extra_fields={"extra_field_key": "extra_field_value_0"}, + ), + StreamSlice( + partition={"first_stream_id": 1, "parent_slice": {}}, + cursor_slice={}, + extra_fields={"extra_field_key": "extra_field_value_1"}, + ), + ], + ) + slicer = CartesianProductStreamSlicer(stream_slicers=[decorated_slicer], parameters={}) + extra_fields = [bool(s.extra_fields) for s in slicer.stream_slices()] + assert len(extra_fields) == 2 + assert all(extra_fields) def test_stream_slices_raises_exception_if_multiple_cursor_slice_components(): diff --git a/unit_tests/sources/declarative/partition_routers/test_grouping_partition_router.py b/unit_tests/sources/declarative/partition_routers/test_grouping_partition_router.py index 9bea606e4..f02cc243d 100644 --- a/unit_tests/sources/declarative/partition_routers/test_grouping_partition_router.py +++ b/unit_tests/sources/declarative/partition_routers/test_grouping_partition_router.py @@ -14,11 +14,20 @@ from airbyte_cdk.sources.declarative.partition_routers.substream_partition_router import ( ParentStreamConfig, ) -from airbyte_cdk.sources.types import StreamSlice +from airbyte_cdk.sources.types import Record, StreamSlice from unit_tests.sources.declarative.partition_routers.test_substream_partition_router import ( MockStream, parent_slices, ) # Reuse MockStream and parent_slices +from unit_tests.sources.streams.concurrent.scenarios.thread_based_concurrent_stream_source_builder import ( + InMemoryPartition, +) + +_EMPTY_SLICE = StreamSlice(partition={}, cursor_slice={}) + + +def _build_records_for_slice(records: List[Mapping[str, Any]], _slice: StreamSlice): + return [Record(record, "stream_name", _slice) for record in records] @pytest.fixture @@ -30,17 +39,29 @@ def mock_config(): def mock_underlying_router(mock_config): """Fixture for a simple underlying router with predefined slices and extra fields.""" parent_stream = MockStream( - slices=[{}], # Single empty slice, parent_partition will be {} - records=[ - {"board_id": 0, "name": "Board 0", "owner": "User0"}, - { - "board_id": 0, - "name": "Board 0 Duplicate", - "owner": "User0 Duplicate", - }, # Duplicate board_id - ] - + [{"board_id": i, "name": f"Board {i}", "owner": f"User{i}"} for i in range(1, 5)], - name="mock_parent", + [ + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [ + {"board_id": 0, "name": "Board 0", "owner": "User0"}, + { + "board_id": 0, + "name": "Board 0 Duplicate", + "owner": "User0 Duplicate", + }, + ] # Duplicate board_id + + [ + {"board_id": i, "name": f"Board {i}", "owner": f"User{i}"} + for i in range(1, 5) + ], + _EMPTY_SLICE, + ), + ) + ], + "first_stream", ) return SubstreamPartitionRouter( parent_stream_configs=[ @@ -62,13 +83,36 @@ def mock_underlying_router(mock_config): def mock_underlying_router_with_parent_slices(mock_config): """Fixture with varied parent slices for testing non-empty parent_slice.""" parent_stream = MockStream( - slices=parent_slices, # [{"slice": "first"}, {"slice": "second"}, {"slice": "third"}] - records=[ - {"board_id": 1, "name": "Board 1", "owner": "User1", "slice": "first"}, - {"board_id": 2, "name": "Board 2", "owner": "User2", "slice": "second"}, - {"board_id": 3, "name": "Board 3", "owner": "User3", "slice": "third"}, + [ + InMemoryPartition( + "partition_1", + "first_stream", + parent_slices[0], + _build_records_for_slice( + [{"board_id": 1, "name": "Board 1", "owner": "User1", "slice": "first"}], + parent_slices[0], + ), + ), + InMemoryPartition( + "partition_2", + "first_stream", + parent_slices[1], + _build_records_for_slice( + [{"board_id": 2, "name": "Board 2", "owner": "User2", "slice": "second"}], + parent_slices[1], + ), + ), + InMemoryPartition( + "partition_3", + "first_stream", + parent_slices[2], + _build_records_for_slice( + [{"board_id": 3, "name": "Board 3", "owner": "User3", "slice": "third"}], + parent_slices[2], + ), + ), ], - name="mock_parent", + "first_stream", ) return SubstreamPartitionRouter( parent_stream_configs=[ @@ -173,9 +217,15 @@ def test_stream_slices_grouping( def test_stream_slices_empty_underlying_router(mock_config): """Test behavior when the underlying router yields no slices.""" parent_stream = MockStream( - slices=[{}], - records=[], - name="mock_parent", + [ + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + [], + ) + ], + "first_stream", ) underlying_router = SubstreamPartitionRouter( parent_stream_configs=[ @@ -298,30 +348,25 @@ def __next__(self): ) -def test_set_initial_state_delegation(mock_config, mock_underlying_router): - """Test that set_initial_state delegates to the underlying router.""" - router = GroupingPartitionRouter( - group_size=2, - underlying_partition_router=mock_underlying_router, - config=mock_config, - ) - mock_state = {"some_key": "some_value"} - mock_underlying_router.set_initial_state = MagicMock() - - router.set_initial_state(mock_state) - mock_underlying_router.set_initial_state.assert_called_once_with(mock_state) - - def test_stream_slices_extra_fields_varied(mock_config): """Test grouping with varied extra fields across partitions.""" parent_stream = MockStream( - slices=[{}], - records=[ - {"board_id": 1, "name": "Board 1", "owner": "User1"}, - {"board_id": 2, "name": "Board 2"}, # Missing owner - {"board_id": 3, "owner": "User3"}, # Missing name + [ + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [ + {"board_id": 1, "name": "Board 1", "owner": "User1"}, + {"board_id": 2, "name": "Board 2"}, # Missing owner + {"board_id": 3, "owner": "User3"}, # Missing name + ], + _EMPTY_SLICE, + ), + ) ], - name="mock_parent", + "first_stream", ) underlying_router = SubstreamPartitionRouter( parent_stream_configs=[ @@ -362,9 +407,21 @@ def test_stream_slices_extra_fields_varied(mock_config): def test_grouping_with_complex_partitions_and_extra_fields(mock_config): """Test grouping with partitions containing multiple keys and extra fields.""" parent_stream = MockStream( - slices=[{}], - records=[{"board_id": i, "extra": f"extra_{i}", "name": f"Board {i}"} for i in range(3)], - name="mock_parent", + [ + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [ + {"board_id": i, "extra": f"extra_{i}", "name": f"Board {i}"} + for i in range(3) + ], + _EMPTY_SLICE, + ), + ) + ], + "first_stream", ) underlying_router = SubstreamPartitionRouter( parent_stream_configs=[ @@ -443,72 +500,3 @@ def test_get_request_params_default(mock_config, mock_underlying_router): ) ) assert params == {} - - -def test_stream_slices_resume_from_state(mock_config, mock_underlying_router): - """Test that stream_slices resumes correctly from a previous state.""" - - # Simulate underlying router state handling - class MockPartitionRouter: - def __init__(self): - self.slices = [ - StreamSlice( - partition={"board_ids": i}, - cursor_slice={}, - extra_fields={"name": f"Board {i}", "owner": f"User{i}"}, - ) - for i in range(5) - ] - self.state = {"last_board_id": 0} # Initial state - - def set_initial_state(self, state): - self.state = state - - def get_stream_state(self): - return self.state - - def stream_slices(self): - last_board_id = self.state.get("last_board_id", -1) - for slice in self.slices: - board_id = slice.partition["board_ids"] - if board_id <= last_board_id: - continue - self.state = {"last_board_id": board_id} - yield slice - - underlying_router = MockPartitionRouter() - router = GroupingPartitionRouter( - group_size=2, - underlying_partition_router=underlying_router, - config=mock_config, - deduplicate=True, - ) - - # First sync: process first two slices - router.set_initial_state({"last_board_id": 0}) - slices_iter = router.stream_slices() - first_batch = next(slices_iter) - assert first_batch == StreamSlice( - partition={"board_ids": [1, 2]}, - cursor_slice={}, - extra_fields={"name": ["Board 1", "Board 2"], "owner": ["User1", "User2"]}, - ) - state_after_first = router.get_stream_state() - assert state_after_first == {"last_board_id": 2}, "State should reflect last processed board_id" - - # Simulate a new sync resuming from the previous state - new_router = GroupingPartitionRouter( - group_size=2, - underlying_partition_router=MockPartitionRouter(), - config=mock_config, - deduplicate=True, - ) - new_router.set_initial_state(state_after_first) - resumed_slices = list(new_router.stream_slices()) - assert resumed_slices == [ - StreamSlice( - partition={"board_ids": [3, 4]}, - cursor_slice={}, - extra_fields={"name": ["Board 3", "Board 4"], "owner": ["User3", "User4"]}, - ) - ], "Should resume from board_id 3" diff --git a/unit_tests/sources/declarative/partition_routers/test_substream_partition_router.py b/unit_tests/sources/declarative/partition_routers/test_substream_partition_router.py index 122c8dfae..feab9bbc0 100644 --- a/unit_tests/sources/declarative/partition_routers/test_substream_partition_router.py +++ b/unit_tests/sources/declarative/partition_routers/test_substream_partition_router.py @@ -3,22 +3,16 @@ # import logging -from functools import partial -from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Union +from typing import Any, Iterable, List, Mapping, Optional +from unittest.mock import Mock import pytest as pytest +from airbyte_protocol_dataclasses.models import AirbyteStream -from airbyte_cdk.legacy.sources.declarative.incremental.per_partition_cursor import ( - CursorFactory, - PerPartitionCursor, -) -from airbyte_cdk.models import AirbyteMessage, AirbyteRecordMessage, SyncMode, Type -from airbyte_cdk.sources.declarative.declarative_stream import DeclarativeStream from airbyte_cdk.sources.declarative.incremental import ( - ChildPartitionResumableFullRefreshCursor, - ResumableFullRefreshCursor, + ConcurrentCursorFactory, + ConcurrentPerPartitionCursor, ) -from airbyte_cdk.sources.declarative.interpolation import InterpolatedString from airbyte_cdk.sources.declarative.partition_routers import ( CartesianProductStreamSlicer, ListPartitionRouter, @@ -32,9 +26,22 @@ RequestOptionType, ) from airbyte_cdk.sources.streams.checkpoint import Cursor +from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream +from airbyte_cdk.sources.streams.concurrent.availability_strategy import StreamAvailability +from airbyte_cdk.sources.streams.concurrent.cursor import ( + ConcurrentCursor, + CursorField, + FinalStateCursor, +) +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.streams.concurrent.state_converters.datetime_stream_state_converter import ( + CustomFormatConcurrentStreamStateConverter, +) from airbyte_cdk.sources.types import Record, StreamSlice -from airbyte_cdk.utils import AirbyteTracedException -from unit_tests.sources.declarative.partition_routers.helpers import MockStream +from airbyte_cdk.utils.datetime_helpers import ab_datetime_parse +from unit_tests.sources.streams.concurrent.scenarios.thread_based_concurrent_stream_source_builder import ( + InMemoryPartition, +) parent_records = [{"id": 1, "data": "data1"}, {"id": 2, "data": "data2"}] more_records = [ @@ -49,91 +56,77 @@ data_second_parent_slice = [{"id": 2, "slice": "second", "data": "C"}] data_third_parent_slice = [] all_parent_data = data_first_parent_slice + data_second_parent_slice + data_third_parent_slice -parent_slices = [{"slice": "first"}, {"slice": "second"}, {"slice": "third"}] +parent_slices = [ + StreamSlice(partition={"slice": "first"}, cursor_slice={}), + StreamSlice(partition={"slice": "second"}, cursor_slice={}), + StreamSlice(partition={"slice": "third"}, cursor_slice={}), +] +parent_slices_with_cursor = [ + StreamSlice( + partition={"slice": "first"}, cursor_slice={"start": "2021-01-01", "end": "2023-01-01"} + ), + StreamSlice( + partition={"slice": "second"}, cursor_slice={"start": "2021-01-01", "end": "2023-01-01"} + ), + StreamSlice( + partition={"slice": "third"}, cursor_slice={"start": "2021-01-01", "end": "2023-01-01"} + ), +] second_parent_stream_slice = [StreamSlice(partition={"slice": "second_parent"}, cursor_slice={})] data_first_parent_slice_with_cursor = [ - {"id": 0, "slice": "first", "data": "A", "cursor": "first_cursor_0"}, - {"id": 1, "slice": "first", "data": "B", "cursor": "first_cursor_1"}, + {"id": 0, "slice": "first", "data": "A", "cursor": "2021-01-01"}, + {"id": 1, "slice": "first", "data": "B", "cursor": "2021-01-02"}, ] data_second_parent_slice_with_cursor = [ - {"id": 2, "slice": "second", "data": "C", "cursor": "second_cursor_2"} + {"id": 2, "slice": "second", "data": "C", "cursor": "2022-01-01"} ] all_parent_data_with_cursor = ( data_first_parent_slice_with_cursor + data_second_parent_slice_with_cursor ) +_EMPTY_SLICE = StreamSlice(partition={}, cursor_slice={}) +_ANY_STREAM = None -class MockIncrementalStream(MockStream): - def __init__(self, slices, records, name, cursor_field="", cursor=None, date_ranges=None): - super().__init__(slices, records, name, cursor_field, cursor) - if date_ranges is None: - date_ranges = [] - self._date_ranges = date_ranges - self._state = {} +def _build_records_for_slice(records: List[Mapping[str, Any]], _slice: StreamSlice): + return [Record(record, "stream_name", _slice) for record in records] - def read_records( - self, - sync_mode: SyncMode, - cursor_field: List[str] = None, - stream_slice: Mapping[str, Any] = None, - stream_state: Mapping[str, Any] = None, - ) -> Iterable[Mapping[str, Any]]: - results = [ - record - for record in self._records - if stream_slice["start_time"] <= record["updated_at"] <= stream_slice["end_time"] - ] - print(f"about to emit {results}") - yield from results - print(f"setting state to {stream_slice}") - self._state = stream_slice +class MockStream(AbstractStream): + def __init__(self, partitions, name, cursor_field="", cursor=None): + self._partitions = partitions + self._stream_cursor_field = cursor_field + self._name = name + self._state = {"states": []} + self._cursor = cursor if cursor else FinalStateCursor(self._name, None, Mock()) -class MockResumableFullRefreshStream(MockStream): - def __init__( - self, - slices, - name, - cursor_field="", - cursor=None, - record_pages: Optional[List[List[Mapping[str, Any]]]] = None, - ): - super().__init__(slices, [], name, cursor_field, cursor) - if record_pages: - self._record_pages = record_pages - else: - self._record_pages = [] - self._state: MutableMapping[str, Any] = {} + def generate_partitions(self) -> Iterable[Partition]: + list(self._cursor.stream_slices()) + return self._partitions - def read_records( - self, - sync_mode: SyncMode, - cursor_field: List[str] = None, - stream_slice: Mapping[str, Any] = None, - stream_state: Mapping[str, Any] = None, - ) -> Iterable[Mapping[str, Any]]: - page_number = self.state.get("next_page_token") or 1 - yield from self._record_pages[page_number - 1] + @property + def name(self) -> str: + return self._name - cursor = self.get_cursor() - if page_number < len(self._record_pages): - cursor.close_slice( - StreamSlice(cursor_slice={"next_page_token": page_number + 1}, partition={}) - ) - else: - cursor.close_slice( - StreamSlice(cursor_slice={"__ab_full_refresh_sync_complete": True}, partition={}) - ) + @property + def cursor_field(self) -> Optional[str]: + return self._stream_cursor_field + + def get_json_schema(self) -> Mapping[str, Any]: + return {} + + def as_airbyte_stream(self) -> AirbyteStream: + raise NotImplementedError() + + def log_stream_sync_configuration(self) -> None: + raise NotImplementedError() @property - def state(self) -> Mapping[str, Any]: - cursor = self.get_cursor() - return cursor.get_stream_state() if cursor else {} + def cursor(self) -> Cursor: + return self._cursor - @state.setter - def state(self, value: Mapping[str, Any]) -> None: - self._state = value + def check_availability(self) -> StreamAvailability: + raise NotImplementedError() @pytest.mark.parametrize( @@ -143,7 +136,10 @@ def state(self, value: Mapping[str, Any]) -> None: ( [ ParentStreamConfig( - stream=MockStream([{}], [], "first_stream"), + stream=MockStream( + [InMemoryPartition("partition_name", "first_stream", _EMPTY_SLICE, [])], + "first_stream", + ), parent_key="id", partition_field="first_stream_id", parameters={}, @@ -155,7 +151,17 @@ def state(self, value: Mapping[str, Any]) -> None: ( [ ParentStreamConfig( - stream=MockStream([{}], parent_records, "first_stream"), + stream=MockStream( + [ + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice(parent_records, _EMPTY_SLICE), + ) + ], + "first_stream", + ), parent_key="id", partition_field="first_stream_id", parameters={}, @@ -170,7 +176,31 @@ def state(self, value: Mapping[str, Any]) -> None: ( [ ParentStreamConfig( - stream=MockStream(parent_slices, all_parent_data, "first_stream"), + stream=MockStream( + [ + InMemoryPartition( + "partition_1", + "first_stream", + parent_slices[0], + _build_records_for_slice(data_first_parent_slice, parent_slices[0]), + ), + InMemoryPartition( + "partition_2", + "first_stream", + parent_slices[1], + _build_records_for_slice( + data_second_parent_slice, parent_slices[1] + ), + ), + InMemoryPartition( + "partition_3", + "first_stream", + parent_slices[2], + _build_records_for_slice(data_third_parent_slice, parent_slices[2]), + ), + ], + "first_stream", + ), parent_key="id", partition_field="first_stream_id", parameters={}, @@ -188,10 +218,27 @@ def state(self, value: Mapping[str, Any]) -> None: ParentStreamConfig( stream=MockStream( [ - StreamSlice(partition=p, cursor_slice={"start": 0, "end": 1}) - for p in parent_slices + InMemoryPartition( + "partition_1", + "first_stream", + parent_slices[0], + _build_records_for_slice(data_first_parent_slice, parent_slices[0]), + ), + InMemoryPartition( + "partition_2", + "first_stream", + parent_slices[1], + _build_records_for_slice( + data_second_parent_slice, parent_slices[1] + ), + ), + InMemoryPartition( + "partition_3", + "first_stream", + parent_slices[2], + _build_records_for_slice(data_third_parent_slice, parent_slices[2]), + ), ], - all_parent_data, "first_stream", ), parent_key="id", @@ -210,8 +257,28 @@ def state(self, value: Mapping[str, Any]) -> None: [ ParentStreamConfig( stream=MockStream( - parent_slices, - data_first_parent_slice + data_second_parent_slice, + [ + InMemoryPartition( + "partition_1", + "first_stream", + parent_slices[0], + _build_records_for_slice(data_first_parent_slice, parent_slices[0]), + ), + InMemoryPartition( + "partition_2", + "first_stream", + parent_slices[1], + _build_records_for_slice( + data_second_parent_slice, parent_slices[1] + ), + ), + InMemoryPartition( + "partition_3", + "first_stream", + parent_slices[2], + [], + ), + ], "first_stream", ), parent_key="id", @@ -220,7 +287,19 @@ def state(self, value: Mapping[str, Any]) -> None: config={}, ), ParentStreamConfig( - stream=MockStream(second_parent_stream_slice, more_records, "second_stream"), + stream=MockStream( + [ + InMemoryPartition( + "partition_1", + "first_stream", + second_parent_stream_slice[0], + _build_records_for_slice( + more_records, second_parent_stream_slice[0] + ), + ), + ], + "first_stream", + ), parent_key="id", partition_field="second_stream_id", parameters={}, @@ -239,7 +318,17 @@ def state(self, value: Mapping[str, Any]) -> None: [ ParentStreamConfig( stream=MockStream( - [{}], [{"id": 0}, {"id": 1}, {"_id": 2}, {"id": 3}], "first_stream" + [ + InMemoryPartition( + "partition_1", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [{"id": 0}, {"id": 1}, {"_id": 2}, {"id": 3}], _EMPTY_SLICE + ), + ), + ], + "first_stream", ), parent_key="id", partition_field="first_stream_id", @@ -257,8 +346,22 @@ def state(self, value: Mapping[str, Any]) -> None: [ ParentStreamConfig( stream=MockStream( - [{}], - [{"a": {"b": 0}}, {"a": {"b": 1}}, {"a": {"c": 2}}, {"a": {"b": 3}}], + [ + InMemoryPartition( + "partition_1", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [ + {"a": {"b": 0}}, + {"a": {"b": 1}}, + {"a": {"c": 2}}, + {"a": {"b": 3}}, + ], + _EMPTY_SLICE, + ), + ), + ], "first_stream", ), parent_key="a/b", @@ -279,8 +382,8 @@ def state(self, value: Mapping[str, Any]) -> None: "test_single_parent_slices_no_records", "test_single_parent_slices_with_records", "test_with_parent_slices_and_records", - "test_multiple_parent_streams", "test_cursor_values_are_removed_from_parent_slices", + "test_multiple_parent_streams", "test_missed_parent_key", "test_dpath_extraction", ], @@ -301,156 +404,6 @@ def test_substream_partition_router(parent_stream_configs, expected_slices): assert slices == expected_slices -def test_substream_partition_router_invalid_parent_record_type(): - partition_router = SubstreamPartitionRouter( - parent_stream_configs=[ - ParentStreamConfig( - stream=MockStream([{}], [list()], "first_stream"), - parent_key="id", - partition_field="first_stream_id", - parameters={}, - config={}, - ) - ], - parameters={}, - config={}, - ) - - with pytest.raises(AirbyteTracedException): - _ = [s for s in partition_router.stream_slices()] - - -@pytest.mark.parametrize( - "initial_state, expected_parent_state", - [ - # Case 1: Empty initial state, no parent state expected - ({}, {}), - # Case 2: Initial state with no `parent_state`, migrate `updated_at` to `parent_stream_cursor` - ( - {"updated_at": "2023-05-27T00:00:00Z"}, - {"parent_stream_cursor": "2023-05-27T00:00:00Z"}, - ), - # Case 3: Initial state with global `state`, no migration expected - ( - {"state": {"updated": "2023-05-27T00:00:00Z"}}, - {"parent_stream_cursor": "2023-05-27T00:00:00Z"}, - ), - # Case 4: Initial state with per-partition `states`, no migration expected - ( - { - "states": [ - { - "partition": { - "issue_id": "10012", - "parent_slice": { - "parent_slice": {}, - "project_id": "10000", - }, - }, - "cursor": {"updated": "2021-01-01T00:00:00+0000"}, - }, - { - "partition": { - "issue_id": "10019", - "parent_slice": { - "parent_slice": {}, - "project_id": "10000", - }, - }, - "cursor": {"updated": "2021-01-01T00:00:00+0000"}, - }, - { - "partition": { - "issue_id": "10000", - "parent_slice": { - "parent_slice": {}, - "project_id": "10000", - }, - }, - "cursor": {"updated": "2021-01-01T00:00:00+0000"}, - }, - ] - }, - {}, - ), - # Case 5: Initial state with `parent_state`, existing parent state persists - ( - { - "parent_state": { - "parent_stream_name1": {"parent_stream_cursor": "2023-05-27T00:00:00Z"}, - }, - }, - {"parent_stream_cursor": "2023-05-27T00:00:00Z"}, - ), - # Case 6: Declarative global cursor state, no migration expected - ( - { - "looback_window": 1, - "use_global_cursor": True, - "state": {"updated": "2023-05-27T00:00:00Z"}, - }, - {"parent_stream_cursor": "2023-05-27T00:00:00Z"}, - ), - # Case 7: Migrate child state to parent state but child state is empty - ( - { - "state": {}, - "states": [], - "parent_state": {"posts": {}}, - "lookback_window": 1, - "use_global_cursor": False, - }, - {}, - ), - ], - ids=[ - "empty_initial_state", - "initial_state_no_parent_legacy_state", - "initial_state_no_parent_global_state", - "initial_state_no_parent_per_partition_state", - "initial_state_with_parent_state", - "initial_state_no_parent_global_state_declarative", - "initial_state_no_parent_and_no_child", - ], -) -def test_set_initial_state(initial_state, expected_parent_state): - """ - Test the `set_initial_state` method of SubstreamPartitionRouter. - - This test verifies that the method correctly handles different initial state formats - and sets the appropriate parent stream state. - """ - parent_stream = MockStream( - slices=[{}], - records=[], - name="parent_stream_name1", - cursor_field="parent_stream_cursor", - ) - parent_stream.state = {} - parent_stream_config = ParentStreamConfig( - stream=parent_stream, - parent_key="id", - partition_field="parent_stream_id", - parameters={}, - config={}, - incremental_dependency=True, - ) - - partition_router = SubstreamPartitionRouter( - parent_stream_configs=[parent_stream_config], - parameters={}, - config={}, - ) - - partition_router.set_initial_state(initial_state) - - # Assert the state of the parent stream - assert parent_stream.state == expected_parent_state, ( - f"Unexpected parent state. Initial state: {initial_state}, " - f"Expected: {expected_parent_state}, Got: {parent_stream.state}" - ) - - @pytest.mark.parametrize( "parent_stream_request_parameters, expected_req_params, expected_headers, expected_body_json, expected_body_data", [ @@ -557,11 +510,7 @@ def test_request_option( partition_router = SubstreamPartitionRouter( parent_stream_configs=[ ParentStreamConfig( - stream=MockStream( - parent_slices, - data_first_parent_slice + data_second_parent_slice, - "first_stream", - ), + stream=_ANY_STREAM, parent_key="id", partition_field="first_stream_id", parameters={}, @@ -569,7 +518,7 @@ def test_request_option( request_option=parent_stream_request_parameters[0], ), ParentStreamConfig( - stream=MockStream(second_parent_stream_slice, more_records, "second_stream"), + stream=_ANY_STREAM, parent_key="id", partition_field="second_stream_id", parameters={}, @@ -594,10 +543,65 @@ def test_request_option( ( ParentStreamConfig( stream=MockStream( - parent_slices, - all_parent_data_with_cursor, + [ + InMemoryPartition( + "partition_1", + "first_stream", + parent_slices_with_cursor[0], + _build_records_for_slice( + data_first_parent_slice_with_cursor, parent_slices_with_cursor[0] + ), + ), + InMemoryPartition( + "partition_2", + "first_stream", + parent_slices_with_cursor[1], + _build_records_for_slice( + data_second_parent_slice_with_cursor, parent_slices_with_cursor[1] + ), + ), + InMemoryPartition( + "partition_3", + "first_stream", + parent_slices_with_cursor[2], + _build_records_for_slice([], parent_slices_with_cursor[2]), + ), + ], "first_stream", - cursor_field="cursor", + cursor=ConcurrentPerPartitionCursor( + cursor_factory=ConcurrentCursorFactory( + lambda stream_state, runtime_lookback_window: ConcurrentCursor( + stream_name="first_stream", + stream_namespace=None, + stream_state=stream_state, + message_repository=Mock(), + connector_state_manager=Mock(), + connector_state_converter=CustomFormatConcurrentStreamStateConverter( + "%Y-%m-%d" + ), + cursor_field=CursorField("cursor"), + slice_boundary_fields=("start", "end"), + start=ab_datetime_parse("2021-01-01").to_datetime(), + end_provider=lambda: ab_datetime_parse("2023-01-01").to_datetime(), + lookback_window=runtime_lookback_window, + ), + ), + partition_router=ListPartitionRouter( + values=["first", "second", "third"], + cursor_field="slice", + config={}, + parameters={}, + ), + stream_name="first_stream", + stream_namespace=None, + stream_state={}, + message_repository=Mock(), + connector_state_manager=Mock(), + connector_state_converter=CustomFormatConcurrentStreamStateConverter( + "%Y-%m-%d" + ), + cursor_field=CursorField("cursor"), + ), ), parent_key="id", partition_field="first_stream_id", @@ -607,10 +611,13 @@ def test_request_option( ), { "first_stream": { + "lookback_window": 0, "states": [ - {"cursor": "first_cursor_1", "partition": "first"}, - {"cursor": "second_cursor_2", "partition": "second"}, - ] + {"cursor": {"cursor": "2021-01-02"}, "partition": {"slice": "first"}}, + {"cursor": {"cursor": "2022-01-01"}, "partition": {"slice": "second"}}, + {"cursor": {"cursor": "2021-01-01"}, "partition": {"slice": "third"}}, + ], + "use_global_cursor": False, } }, ), @@ -702,59 +709,6 @@ def test_request_params_interpolation_for_parent_stream( assert partition_router.get_request_params(stream_slice=stream_slice) == expected_request_params -def test_given_record_is_airbyte_message_when_stream_slices_then_use_record_data(): - parent_slice = {} - partition_router = SubstreamPartitionRouter( - parent_stream_configs=[ - ParentStreamConfig( - stream=MockStream( - [parent_slice], - [ - AirbyteMessage( - type=Type.RECORD, - record=AirbyteRecordMessage( - data={"id": "record value"}, emitted_at=0, stream="stream" - ), - ) - ], - "first_stream", - ), - parent_key="id", - partition_field="partition_field", - parameters={}, - config={}, - ) - ], - parameters={}, - config={}, - ) - - slices = list(partition_router.stream_slices()) - assert slices == [{"partition_field": "record value", "parent_slice": parent_slice}] - - -def test_given_record_is_record_object_when_stream_slices_then_use_record_data(): - parent_slice = {} - partition_router = SubstreamPartitionRouter( - parent_stream_configs=[ - ParentStreamConfig( - stream=MockStream( - [parent_slice], [Record({"id": "record value"}, {})], "first_stream" - ), - parent_key="id", - partition_field="partition_field", - parameters={}, - config={}, - ) - ], - parameters={}, - config={}, - ) - - slices = list(partition_router.stream_slices()) - assert slices == [{"partition_field": "record value", "parent_slice": parent_slice}] - - def test_substream_using_incremental_parent_stream(): mock_slices = [ StreamSlice( @@ -775,15 +729,44 @@ def test_substream_using_incremental_parent_stream(): partition_router = SubstreamPartitionRouter( parent_stream_configs=[ ParentStreamConfig( - stream=MockIncrementalStream( - slices=mock_slices, - records=[ - Record({"id": "may_record_0", "updated_at": "2024-05-15"}, mock_slices[0]), - Record({"id": "may_record_1", "updated_at": "2024-05-16"}, mock_slices[0]), - Record({"id": "jun_record_0", "updated_at": "2024-06-15"}, mock_slices[1]), - Record({"id": "jun_record_1", "updated_at": "2024-06-16"}, mock_slices[1]), + stream=MockStream( + [ + InMemoryPartition( + "partition_1", + "first_stream", + mock_slices[0], + [ + Record( + {"id": "may_record_0", "updated_at": "2024-05-15"}, + "first_stream", + mock_slices[0], + ), + Record( + {"id": "may_record_1", "updated_at": "2024-05-16"}, + "first_stream", + mock_slices[0], + ), + ], + ), + InMemoryPartition( + "partition_1", + "first_stream", + mock_slices[1], + [ + Record( + {"id": "jun_record_0", "updated_at": "2024-06-15"}, + "first_stream", + mock_slices[1], + ), + Record( + {"id": "jun_record_1", "updated_at": "2024-06-16"}, + "first_stream", + mock_slices[1], + ), + ], + ), ], - name="first_stream", + "first_stream", ), parent_key="id", partition_field="partition_field", @@ -822,25 +805,69 @@ def test_substream_checkpoints_after_each_parent_partition(): ] expected_parent_state = [ - {"first_stream": {}}, - {"first_stream": {}}, - {"first_stream": {"start_time": "2024-04-27", "end_time": "2024-05-27"}}, - {"first_stream": {"start_time": "2024-04-27", "end_time": "2024-05-27"}}, - {"first_stream": {"start_time": "2024-05-27", "end_time": "2024-06-27"}}, + {"first_stream": {"updated_at": mock_slices[0]["start_time"]}}, + {"first_stream": {"updated_at": "2024-05-16"}}, + {"first_stream": {"updated_at": "2024-05-16"}}, + {"first_stream": {"updated_at": "2024-06-16"}}, + {"first_stream": {"updated_at": "2024-06-16"}}, ] partition_router = SubstreamPartitionRouter( parent_stream_configs=[ ParentStreamConfig( - stream=MockIncrementalStream( - slices=mock_slices, - records=[ - Record({"id": "may_record_0", "updated_at": "2024-05-15"}, mock_slices[0]), - Record({"id": "may_record_1", "updated_at": "2024-05-16"}, mock_slices[0]), - Record({"id": "jun_record_0", "updated_at": "2024-06-15"}, mock_slices[1]), - Record({"id": "jun_record_1", "updated_at": "2024-06-16"}, mock_slices[1]), + stream=MockStream( + [ + InMemoryPartition( + "partition_1", + "first_stream", + mock_slices[0], + [ + Record( + {"id": "may_record_0", "updated_at": "2024-05-15"}, + "first_stream", + mock_slices[0], + ), + Record( + {"id": "may_record_1", "updated_at": "2024-05-16"}, + "first_stream", + mock_slices[0], + ), + ], + ), + InMemoryPartition( + "partition_1", + "first_stream", + mock_slices[1], + [ + Record( + {"id": "jun_record_0", "updated_at": "2024-06-15"}, + "first_stream", + mock_slices[1], + ), + Record( + {"id": "jun_record_1", "updated_at": "2024-06-16"}, + "first_stream", + mock_slices[1], + ), + ], + ), ], - name="first_stream", + "first_stream", + "updated_at", + ConcurrentCursor( + stream_name="first_stream", + stream_namespace=None, + stream_state={}, + message_repository=Mock(), + connector_state_manager=Mock(), + connector_state_converter=CustomFormatConcurrentStreamStateConverter( + "%Y-%m-%d" + ), + cursor_field=CursorField("updated_at"), + slice_boundary_fields=("start_time", "end_time"), + start=ab_datetime_parse(mock_slices[0]["start_time"]).to_datetime(), + end_provider=lambda: ab_datetime_parse("2023-01-01").to_datetime(), + ), ), incremental_dependency=True, parent_key="id", @@ -858,273 +885,7 @@ def test_substream_checkpoints_after_each_parent_partition(): assert actual_slice == expected_slices[expected_counter] assert partition_router.get_stream_state() == expected_parent_state[expected_counter] expected_counter += 1 - assert partition_router.get_stream_state() == expected_parent_state[expected_counter] - - -@pytest.mark.parametrize( - "use_incremental_dependency", - [ - pytest.param(False, id="test_resumable_full_refresh_stream_without_parent_checkpoint"), - pytest.param( - True, - id="test_resumable_full_refresh_stream_with_use_incremental_dependency_for_parent_checkpoint", - ), - ], -) -def test_substream_using_resumable_full_refresh_parent_stream(use_incremental_dependency): - mock_slices = [ - StreamSlice(cursor_slice={}, partition={}), - StreamSlice(cursor_slice={"next_page_token": 2}, partition={}), - StreamSlice(cursor_slice={"next_page_token": 3}, partition={}), - ] - - expected_slices = [ - {"partition_field": "makoto_yuki", "parent_slice": {}}, - {"partition_field": "yukari_takeba", "parent_slice": {}}, - {"partition_field": "mitsuru_kirijo", "parent_slice": {}}, - {"partition_field": "akihiko_sanada", "parent_slice": {}}, - {"partition_field": "junpei_iori", "parent_slice": {}}, - {"partition_field": "fuuka_yamagishi", "parent_slice": {}}, - ] - - expected_parent_state = [ - {"persona_3_characters": {}}, - {"persona_3_characters": {}}, - {"persona_3_characters": {"next_page_token": 2}}, - {"persona_3_characters": {"next_page_token": 2}}, - {"persona_3_characters": {"next_page_token": 3}}, - {"persona_3_characters": {"next_page_token": 3}}, - {"persona_3_characters": {"__ab_full_refresh_sync_complete": True}}, - ] - - partition_router = SubstreamPartitionRouter( - parent_stream_configs=[ - ParentStreamConfig( - stream=MockResumableFullRefreshStream( - slices=[StreamSlice(partition={}, cursor_slice={})], - cursor=ResumableFullRefreshCursor(parameters={}), - record_pages=[ - [ - Record( - data={"id": "makoto_yuki"}, - associated_slice=mock_slices[0], - stream_name="test_stream", - ), - Record( - data={"id": "yukari_takeba"}, - associated_slice=mock_slices[0], - stream_name="test_stream", - ), - ], - [ - Record( - data={"id": "mitsuru_kirijo"}, - associated_slice=mock_slices[1], - stream_name="test_stream", - ), - Record( - data={"id": "akihiko_sanada"}, - associated_slice=mock_slices[1], - stream_name="test_stream", - ), - ], - [ - Record( - data={"id": "junpei_iori"}, - associated_slice=mock_slices[2], - stream_name="test_stream", - ), - Record( - data={"id": "fuuka_yamagishi"}, - associated_slice=mock_slices[2], - stream_name="test_stream", - ), - ], - ], - name="persona_3_characters", - ), - incremental_dependency=use_incremental_dependency, - parent_key="id", - partition_field="partition_field", - parameters={}, - config={}, - ) - ], - parameters={}, - config={}, - ) - - expected_counter = 0 - for actual_slice in partition_router.stream_slices(): - assert actual_slice == expected_slices[expected_counter] - if use_incremental_dependency: - assert partition_router.get_stream_state() == expected_parent_state[expected_counter] - expected_counter += 1 - if use_incremental_dependency: - assert partition_router.get_stream_state() == expected_parent_state[expected_counter] - - -@pytest.mark.parametrize( - "use_incremental_dependency", - [ - pytest.param( - False, id="test_substream_resumable_full_refresh_stream_without_parent_checkpoint" - ), - pytest.param( - True, - id="test_substream_resumable_full_refresh_stream_with_use_incremental_dependency_for_parent_checkpoint", - ), - ], -) -def test_substream_using_resumable_full_refresh_parent_stream_slices(use_incremental_dependency): - mock_parent_slices = [ - StreamSlice(cursor_slice={}, partition={}), - StreamSlice(cursor_slice={"next_page_token": 2}, partition={}), - StreamSlice(cursor_slice={"next_page_token": 3}, partition={}), - ] - - expected_parent_slices = [ - {"partition_field": "makoto_yuki", "parent_slice": {}}, - {"partition_field": "yukari_takeba", "parent_slice": {}}, - {"partition_field": "mitsuru_kirijo", "parent_slice": {}}, - {"partition_field": "akihiko_sanada", "parent_slice": {}}, - {"partition_field": "junpei_iori", "parent_slice": {}}, - {"partition_field": "fuuka_yamagishi", "parent_slice": {}}, - ] - - expected_parent_state = [ - {"persona_3_characters": {}}, - {"persona_3_characters": {}}, - {"persona_3_characters": {"next_page_token": 2}}, - {"persona_3_characters": {"next_page_token": 2}}, - {"persona_3_characters": {"next_page_token": 3}}, - {"persona_3_characters": {"next_page_token": 3}}, - {"persona_3_characters": {"__ab_full_refresh_sync_complete": True}}, - ] - - expected_substream_state = { - "states": [ - { - "partition": {"parent_slice": {}, "partition_field": "makoto_yuki"}, - "cursor": {"__ab_full_refresh_sync_complete": True}, - }, - { - "partition": {"parent_slice": {}, "partition_field": "yukari_takeba"}, - "cursor": {"__ab_full_refresh_sync_complete": True}, - }, - { - "partition": {"parent_slice": {}, "partition_field": "mitsuru_kirijo"}, - "cursor": {"__ab_full_refresh_sync_complete": True}, - }, - { - "partition": {"parent_slice": {}, "partition_field": "akihiko_sanada"}, - "cursor": {"__ab_full_refresh_sync_complete": True}, - }, - { - "partition": {"parent_slice": {}, "partition_field": "junpei_iori"}, - "cursor": {"__ab_full_refresh_sync_complete": True}, - }, - { - "partition": {"parent_slice": {}, "partition_field": "fuuka_yamagishi"}, - "cursor": {"__ab_full_refresh_sync_complete": True}, - }, - ], - "parent_state": {"persona_3_characters": {"__ab_full_refresh_sync_complete": True}}, - } - - partition_router = SubstreamPartitionRouter( - parent_stream_configs=[ - ParentStreamConfig( - stream=MockResumableFullRefreshStream( - slices=[StreamSlice(partition={}, cursor_slice={})], - cursor=ResumableFullRefreshCursor(parameters={}), - record_pages=[ - [ - Record( - data={"id": "makoto_yuki"}, - associated_slice=mock_parent_slices[0], - stream_name="test_stream", - ), - Record( - data={"id": "yukari_takeba"}, - associated_slice=mock_parent_slices[0], - stream_name="test_stream", - ), - ], - [ - Record( - data={"id": "mitsuru_kirijo"}, - associated_slice=mock_parent_slices[1], - stream_name="test_stream", - ), - Record( - data={"id": "akihiko_sanada"}, - associated_slice=mock_parent_slices[1], - stream_name="test_stream", - ), - ], - [ - Record( - data={"id": "junpei_iori"}, - associated_slice=mock_parent_slices[2], - stream_name="test_stream", - ), - Record( - data={"id": "fuuka_yamagishi"}, - associated_slice=mock_parent_slices[2], - stream_name="test_stream", - ), - ], - ], - name="persona_3_characters", - ), - incremental_dependency=use_incremental_dependency, - parent_key="id", - partition_field="partition_field", - parameters={}, - config={}, - ) - ], - parameters={}, - config={}, - ) - - substream_cursor_slicer = PerPartitionCursor( - cursor_factory=CursorFactory( - create_function=partial(ChildPartitionResumableFullRefreshCursor, {}) - ), - partition_router=partition_router, - ) - - expected_counter = 0 - for actual_slice in substream_cursor_slicer.stream_slices(): - # close the substream slice - substream_cursor_slicer.close_slice(actual_slice) - # check the slice has been processed - assert actual_slice == expected_parent_slices[expected_counter] - # check for parent state - if use_incremental_dependency: - assert ( - substream_cursor_slicer._partition_router.get_stream_state() - == expected_parent_state[expected_counter] - ) - expected_counter += 1 - if use_incremental_dependency: - assert ( - substream_cursor_slicer._partition_router.get_stream_state() - == expected_parent_state[expected_counter] - ) - - # validate final state for closed substream slices - final_state = substream_cursor_slicer.get_stream_state() - if not use_incremental_dependency: - assert final_state["states"] == expected_substream_state["states"], ( - "State for substreams is not valid!" - ) - else: - assert final_state == expected_substream_state, ( - "State for substreams with incremental dependency is not valid!" - ) + assert partition_router.get_stream_state() == expected_parent_state[-1] @pytest.mark.parametrize( @@ -1134,18 +895,27 @@ def test_substream_using_resumable_full_refresh_parent_stream_slices(use_increme [ ParentStreamConfig( stream=MockStream( - [{}], [ - { - "id": 1, - "field_1": "value_1", - "field_2": {"nested_field": "nested_value_1"}, - }, - { - "id": 2, - "field_1": "value_2", - "field_2": {"nested_field": "nested_value_2"}, - }, + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [ + { + "id": 1, + "field_1": "value_1", + "field_2": {"nested_field": "nested_value_1"}, + }, + { + "id": 2, + "field_1": "value_2", + "field_2": {"nested_field": "nested_value_2"}, + }, + ], + _EMPTY_SLICE, + ), + ) ], "first_stream", ), @@ -1165,8 +935,20 @@ def test_substream_using_resumable_full_refresh_parent_stream_slices(use_increme [ ParentStreamConfig( stream=MockStream( - [{}], - [{"id": 1, "field_1": "value_1"}, {"id": 2, "field_1": "value_2"}], + [ + InMemoryPartition( + "partition_name", + "first_stream", + _EMPTY_SLICE, + _build_records_for_slice( + [ + {"id": 1, "field_1": "value_1"}, + {"id": 2, "field_1": "value_2"}, + ], + _EMPTY_SLICE, + ), + ) + ], "first_stream", ), parent_key="id", diff --git a/unit_tests/sources/declarative/requesters/request_options/test_per_partition_request_option_provider.py b/unit_tests/sources/declarative/requesters/request_options/test_per_partition_request_option_provider.py new file mode 100644 index 000000000..598aa92ac --- /dev/null +++ b/unit_tests/sources/declarative/requesters/request_options/test_per_partition_request_option_provider.py @@ -0,0 +1,230 @@ +from unittest import TestCase +from unittest.mock import Mock + +from airbyte_cdk.sources.declarative.partition_routers import PartitionRouter +from airbyte_cdk.sources.declarative.requesters.request_options import RequestOptionsProvider +from airbyte_cdk.sources.declarative.requesters.request_options.per_partition_request_option_provider import ( + PerPartitionRequestOptionsProvider, +) +from airbyte_cdk.sources.declarative.types import StreamSlice + +_STREAM_STATE = {"state_key": "state_value"} +_STREAM_SLICE = StreamSlice(partition={"slice_key": "slice_value"}, cursor_slice={}) +_NEXT_PAGE_TOKEN = {"page_token_key": "page_token_value"} + + +class TestPerPartitionRequestOptionsProvider(TestCase): + def setUp(self): + self._partition_router = Mock(spec=PartitionRouter) + self._cursor_provider = Mock(spec=RequestOptionsProvider) + self._option_provider = PerPartitionRequestOptionsProvider( + self._partition_router, self._cursor_provider + ) + + def test_given_partition_router_value_when_get_request_params_then_return_partition_router_params( + self, + ) -> None: + self._partition_router.get_request_params.return_value = {"key": "value"} + self._cursor_provider.get_request_params.return_value = dict() + + result = self._option_provider.get_request_params( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_cursor_provider_value_when_get_request_params_then_return_partition_router_params( + self, + ) -> None: + self._partition_router.get_request_params.return_value = dict() + self._cursor_provider.get_request_params.return_value = {"key": "value"} + + result = self._option_provider.get_request_params( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_both_provide_value_when_get_request_params_then_overwrite_from_cursor( + self, + ) -> None: + self._partition_router.get_request_params.return_value = { + "key_duplicate": "value_partition", + "key_partition": "value_partition", + } + self._cursor_provider.get_request_params.return_value = { + "key_duplicate": "value_cursor", + "key_cursor": "value_cursor", + } + + result = self._option_provider.get_request_params( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == { + "key_duplicate": "value_cursor", + "key_partition": "value_partition", + "key_cursor": "value_cursor", + } + + def test_given_partition_router_value_when_get_request_headers_then_return_partition_router_headers( + self, + ) -> None: + self._partition_router.get_request_headers.return_value = {"key": "value"} + self._cursor_provider.get_request_headers.return_value = dict() + + result = self._option_provider.get_request_headers( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_cursor_provider_value_when_get_request_headers_then_return_cursor_provider_headers( + self, + ) -> None: + self._partition_router.get_request_headers.return_value = dict() + self._cursor_provider.get_request_headers.return_value = {"key": "value"} + + result = self._option_provider.get_request_headers( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_both_provide_value_when_get_request_headers_then_overwrite_from_cursor( + self, + ) -> None: + self._partition_router.get_request_headers.return_value = { + "key_duplicate": "value_partition", + "key_partition": "value_partition", + } + self._cursor_provider.get_request_headers.return_value = { + "key_duplicate": "value_cursor", + "key_cursor": "value_cursor", + } + + result = self._option_provider.get_request_headers( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == { + "key_duplicate": "value_cursor", + "key_partition": "value_partition", + "key_cursor": "value_cursor", + } + + def test_given_partition_router_value_when_get_request_body_data_then_return_partition_router_body_data( + self, + ) -> None: + self._partition_router.get_request_body_data.return_value = {"key": "value"} + self._cursor_provider.get_request_body_data.return_value = dict() + + result = self._option_provider.get_request_body_data( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_cursor_provider_value_when_get_request_body_data_then_return_cursor_provider_body_data( + self, + ) -> None: + self._partition_router.get_request_body_data.return_value = dict() + self._cursor_provider.get_request_body_data.return_value = {"key": "value"} + + result = self._option_provider.get_request_body_data( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_both_provide_value_when_get_request_body_data_then_overwrite_from_cursor( + self, + ) -> None: + self._partition_router.get_request_body_data.return_value = { + "key_duplicate": "value_partition", + "key_partition": "value_partition", + } + self._cursor_provider.get_request_body_data.return_value = { + "key_duplicate": "value_cursor", + "key_cursor": "value_cursor", + } + + result = self._option_provider.get_request_body_data( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == { + "key_duplicate": "value_cursor", + "key_partition": "value_partition", + "key_cursor": "value_cursor", + } + + def test_given_partition_router_value_when_get_request_body_json_then_return_partition_router_body_json( + self, + ) -> None: + self._partition_router.get_request_body_json.return_value = {"key": "value"} + self._cursor_provider.get_request_body_json.return_value = dict() + + result = self._option_provider.get_request_body_json( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_cursor_provider_value_when_get_request_body_json_then_return_cursor_provider_body_json( + self, + ) -> None: + self._partition_router.get_request_body_json.return_value = dict() + self._cursor_provider.get_request_body_json.return_value = {"key": "value"} + + result = self._option_provider.get_request_body_json( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == {"key": "value"} + + def test_given_both_provide_value_when_get_request_body_json_then_overwrite_from_cursor( + self, + ) -> None: + self._partition_router.get_request_body_json.return_value = { + "key_duplicate": "value_partition", + "key_partition": "value_partition", + } + self._cursor_provider.get_request_body_json.return_value = { + "key_duplicate": "value_cursor", + "key_cursor": "value_cursor", + } + + result = self._option_provider.get_request_body_json( + stream_state=_STREAM_STATE, + stream_slice=_STREAM_SLICE, + next_page_token=_NEXT_PAGE_TOKEN, + ) + + assert result == { + "key_duplicate": "value_cursor", + "key_partition": "value_partition", + "key_cursor": "value_cursor", + } diff --git a/unit_tests/sources/declarative/resolvers/test_http_components_resolver.py b/unit_tests/sources/declarative/resolvers/test_http_components_resolver.py index 437822181..991539e1e 100644 --- a/unit_tests/sources/declarative/resolvers/test_http_components_resolver.py +++ b/unit_tests/sources/declarative/resolvers/test_http_components_resolver.py @@ -392,11 +392,13 @@ def test_http_components_resolver( ): mock_retriever = MagicMock() mock_retriever.read_records.return_value = retriever_data - mock_retriever.stream_slices.return_value = [{}] + stream_slicer = MagicMock() + stream_slicer.stream_slices.return_value = [{}] config = {} resolver = HttpComponentsResolver( retriever=mock_retriever, + stream_slicer=stream_slicer, config=config, components_mapping=components_mapping, parameters={}, @@ -457,11 +459,13 @@ def test_http_components_resolver_with_stream_slices( ): mock_retriever = MagicMock() mock_retriever.read_records.return_value = retriever_data - mock_retriever.stream_slices.return_value = [{"parent_id": 1}, {"parent_id": 2}] + stream_slicer = MagicMock() + stream_slicer.stream_slices.return_value = [{"parent_id": 1}, {"parent_id": 2}] config = {} resolver = HttpComponentsResolver( retriever=mock_retriever, + stream_slicer=stream_slicer, config=config, components_mapping=components_mapping, parameters={}, diff --git a/unit_tests/sources/declarative/test_concurrent_declarative_source.py b/unit_tests/sources/declarative/test_concurrent_declarative_source.py index 8fadc04fd..72152a167 100644 --- a/unit_tests/sources/declarative/test_concurrent_declarative_source.py +++ b/unit_tests/sources/declarative/test_concurrent_declarative_source.py @@ -4046,14 +4046,12 @@ def test_only_parent_streams_use_cache(): # Parent stream created for substream assert ( - stream_1._stream_partition_generator._partition_factory._retriever.stream_slicer.parent_stream_configs[ - 0 - ].stream.name + stream_1._stream_partition_generator._stream_slicer.parent_stream_configs[0].stream.name == "applications" ) - assert stream_1._stream_partition_generator._partition_factory._retriever.stream_slicer.parent_stream_configs[ + assert stream_1._stream_partition_generator._stream_slicer.parent_stream_configs[ 0 - ].stream.retriever.requester.use_cache + ].stream._stream_partition_generator._partition_factory._retriever.requester.use_cache # Main stream without caching stream_2 = streams[2] diff --git a/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py b/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py index b0dcd272c..48816d39b 100644 --- a/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py +++ b/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py @@ -125,7 +125,6 @@ def __init__(self, name, stream_name, _slice, records): self._stream_name = stream_name self._slice = _slice self._records = records - self._is_closed = False def read(self) -> Iterable[Record]: for record_or_exception in self._records: @@ -145,12 +144,6 @@ def __hash__(self) -> int: else: return hash(self._name) - def close(self) -> None: - self._is_closed = True - - def is_closed(self) -> bool: - return self._is_closed - class ConcurrentSourceBuilder(SourceBuilder[ConcurrentCdkSource]): def __init__(self):