diff --git a/airbyte_cdk/connector_builder/connector_builder_handler.py b/airbyte_cdk/connector_builder/connector_builder_handler.py index 513546737..2878a3fda 100644 --- a/airbyte_cdk/connector_builder/connector_builder_handler.py +++ b/airbyte_cdk/connector_builder/connector_builder_handler.py @@ -3,8 +3,8 @@ # -from dataclasses import asdict, dataclass, field -from typing import Any, ClassVar, Dict, List, Mapping +from dataclasses import asdict +from typing import Any, Dict, List, Mapping, Optional from airbyte_cdk.connector_builder.test_reader import TestReader from airbyte_cdk.models import ( @@ -15,45 +15,32 @@ Type, ) from airbyte_cdk.models import Type as MessageType +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, + TestLimits, +) from airbyte_cdk.sources.declarative.declarative_source import DeclarativeSource from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource -from airbyte_cdk.sources.declarative.parsers.model_to_component_factory import ( - ModelToComponentFactory, -) from airbyte_cdk.utils.airbyte_secrets_utils import filter_secrets from airbyte_cdk.utils.datetime_helpers import ab_datetime_now from airbyte_cdk.utils.traced_exception import AirbyteTracedException -DEFAULT_MAXIMUM_NUMBER_OF_PAGES_PER_SLICE = 5 -DEFAULT_MAXIMUM_NUMBER_OF_SLICES = 5 -DEFAULT_MAXIMUM_RECORDS = 100 -DEFAULT_MAXIMUM_STREAMS = 100 - MAX_PAGES_PER_SLICE_KEY = "max_pages_per_slice" MAX_SLICES_KEY = "max_slices" MAX_RECORDS_KEY = "max_records" MAX_STREAMS_KEY = "max_streams" -@dataclass -class TestLimits: - __test__: ClassVar[bool] = False # Tell Pytest this is not a Pytest class, despite its name - - max_records: int = field(default=DEFAULT_MAXIMUM_RECORDS) - max_pages_per_slice: int = field(default=DEFAULT_MAXIMUM_NUMBER_OF_PAGES_PER_SLICE) - max_slices: int = field(default=DEFAULT_MAXIMUM_NUMBER_OF_SLICES) - max_streams: int = field(default=DEFAULT_MAXIMUM_STREAMS) - - def get_limits(config: Mapping[str, Any]) -> TestLimits: command_config = config.get("__test_read_config", {}) - max_pages_per_slice = ( - command_config.get(MAX_PAGES_PER_SLICE_KEY) or DEFAULT_MAXIMUM_NUMBER_OF_PAGES_PER_SLICE + return TestLimits( + max_records=command_config.get(MAX_RECORDS_KEY, TestLimits.DEFAULT_MAX_RECORDS), + max_pages_per_slice=command_config.get( + MAX_PAGES_PER_SLICE_KEY, TestLimits.DEFAULT_MAX_PAGES_PER_SLICE + ), + max_slices=command_config.get(MAX_SLICES_KEY, TestLimits.DEFAULT_MAX_SLICES), + max_streams=command_config.get(MAX_STREAMS_KEY, TestLimits.DEFAULT_MAX_STREAMS), ) - max_slices = command_config.get(MAX_SLICES_KEY) or DEFAULT_MAXIMUM_NUMBER_OF_SLICES - max_records = command_config.get(MAX_RECORDS_KEY) or DEFAULT_MAXIMUM_RECORDS - max_streams = command_config.get(MAX_STREAMS_KEY) or DEFAULT_MAXIMUM_STREAMS - return TestLimits(max_records, max_pages_per_slice, max_slices, max_streams) def should_migrate_manifest(config: Mapping[str, Any]) -> bool: @@ -75,21 +62,28 @@ def should_normalize_manifest(config: Mapping[str, Any]) -> bool: return config.get("__should_normalize", False) -def create_source(config: Mapping[str, Any], limits: TestLimits) -> ManifestDeclarativeSource: +def create_source( + config: Mapping[str, Any], + limits: TestLimits, + catalog: Optional[ConfiguredAirbyteCatalog], + state: Optional[List[AirbyteStateMessage]], +) -> ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]]: manifest = config["__injected_declarative_manifest"] - return ManifestDeclarativeSource( + + # We enforce a concurrency level of 1 so that the stream is processed on a single thread + # to retain ordering for the grouping of the builder message responses. + if "concurrency_level" in manifest: + manifest["concurrency_level"]["default_concurrency"] = 1 + else: + manifest["concurrency_level"] = {"type": "ConcurrencyLevel", "default_concurrency": 1} + + return ConcurrentDeclarativeSource( + catalog=catalog, config=config, - emit_connector_builder_messages=True, + state=state, source_config=manifest, - migrate_manifest=should_migrate_manifest(config), - normalize_manifest=should_normalize_manifest(config), - component_factory=ModelToComponentFactory( - emit_connector_builder_messages=True, - limit_pages_fetched_per_slice=limits.max_pages_per_slice, - limit_slices_fetched=limits.max_slices, - disable_retries=True, - disable_cache=True, - ), + emit_connector_builder_messages=True, + limits=limits, ) diff --git a/airbyte_cdk/connector_builder/main.py b/airbyte_cdk/connector_builder/main.py index 80cf4afa9..22be81c82 100644 --- a/airbyte_cdk/connector_builder/main.py +++ b/airbyte_cdk/connector_builder/main.py @@ -91,12 +91,12 @@ def handle_connector_builder_request( def handle_request(args: List[str]) -> str: command, config, catalog, state = get_config_and_catalog_from_args(args) limits = get_limits(config) - source = create_source(config, limits) - return orjson.dumps( + source = create_source(config=config, limits=limits, catalog=catalog, state=state) + return orjson.dumps( # type: ignore[no-any-return] # Serializer.dump() always returns AirbyteMessage AirbyteMessageSerializer.dump( handle_connector_builder_request(source, command, config, catalog, state, limits) ) - ).decode() # type: ignore[no-any-return] # Serializer.dump() always returns AirbyteMessage + ).decode() if __name__ == "__main__": diff --git a/airbyte_cdk/connector_builder/test_reader/helpers.py b/airbyte_cdk/connector_builder/test_reader/helpers.py index 9154610cc..3cc634ccb 100644 --- a/airbyte_cdk/connector_builder/test_reader/helpers.py +++ b/airbyte_cdk/connector_builder/test_reader/helpers.py @@ -5,7 +5,7 @@ import json from copy import deepcopy from json import JSONDecodeError -from typing import Any, Dict, List, Mapping, Optional +from typing import Any, Dict, List, Mapping, Optional, Union from airbyte_cdk.connector_builder.models import ( AuxiliaryRequest, @@ -17,6 +17,8 @@ from airbyte_cdk.models import ( AirbyteLogMessage, AirbyteMessage, + AirbyteStateBlob, + AirbyteStateMessage, OrchestratorType, TraceType, ) @@ -466,7 +468,7 @@ def handle_current_slice( return StreamReadSlices( pages=current_slice_pages, slice_descriptor=current_slice_descriptor, - state=[latest_state_message] if latest_state_message else [], + state=[convert_state_blob_to_mapping(latest_state_message)] if latest_state_message else [], auxiliary_requests=auxiliary_requests if auxiliary_requests else [], ) @@ -718,3 +720,23 @@ def get_auxiliary_request_type(stream: dict, http: dict) -> str: # type: ignore Determines the type of the auxiliary request based on the stream and HTTP properties. """ return "PARENT_STREAM" if stream.get("is_substream", False) else str(http.get("type", None)) + + +def convert_state_blob_to_mapping( + state_message: Union[AirbyteStateMessage, Dict[str, Any]], +) -> Dict[str, Any]: + """ + The AirbyteStreamState stores state as an AirbyteStateBlob which deceivingly is not + a dictionary, but rather a list of kwargs fields. This in turn causes it to not be + properly turned into a dictionary when translating this back into response output + by the connector_builder_handler using asdict() + """ + + if isinstance(state_message, AirbyteStateMessage) and state_message.stream: + state_value = state_message.stream.stream_state + if isinstance(state_value, AirbyteStateBlob): + state_value_mapping = {k: v for k, v in state_value.__dict__.items()} + state_message.stream.stream_state = state_value_mapping # type: ignore # we intentionally set this as a Dict so that StreamReadSlices is translated properly in the resulting HTTP response + return state_message # type: ignore # See above, but when this is an AirbyteStateMessage we must convert AirbyteStateBlob to a Dict + else: + return state_message # type: ignore # This is guaranteed to be a Dict since we check isinstance AirbyteStateMessage above diff --git a/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py b/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py index 09bd921e1..33731e74c 100644 --- a/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py +++ b/airbyte_cdk/sources/concurrent_source/concurrent_read_processor.py @@ -95,11 +95,14 @@ def on_partition(self, partition: Partition) -> None: """ stream_name = partition.stream_name() self._streams_to_running_partitions[stream_name].add(partition) + cursor = self._stream_name_to_instance[stream_name].cursor if self._slice_logger.should_log_slice_message(self._logger): self._message_repository.emit_message( self._slice_logger.create_slice_log_message(partition.to_slice()) ) - self._thread_pool_manager.submit(self._partition_reader.process_partition, partition) + self._thread_pool_manager.submit( + self._partition_reader.process_partition, partition, cursor + ) def on_partition_complete_sentinel( self, sentinel: PartitionCompleteSentinel @@ -112,26 +115,16 @@ def on_partition_complete_sentinel( """ partition = sentinel.partition - try: - if sentinel.is_successful: - stream = self._stream_name_to_instance[partition.stream_name()] - stream.cursor.close_partition(partition) - except Exception as exception: - self._flag_exception(partition.stream_name(), exception) - yield AirbyteTracedException.from_exception( - exception, stream_descriptor=StreamDescriptor(name=partition.stream_name()) - ).as_sanitized_airbyte_message() - finally: - partitions_running = self._streams_to_running_partitions[partition.stream_name()] - if partition in partitions_running: - partitions_running.remove(partition) - # If all partitions were generated and this was the last one, the stream is done - if ( - partition.stream_name() not in self._streams_currently_generating_partitions - and len(partitions_running) == 0 - ): - yield from self._on_stream_is_done(partition.stream_name()) - yield from self._message_repository.consume_queue() + partitions_running = self._streams_to_running_partitions[partition.stream_name()] + if partition in partitions_running: + partitions_running.remove(partition) + # If all partitions were generated and this was the last one, the stream is done + if ( + partition.stream_name() not in self._streams_currently_generating_partitions + and len(partitions_running) == 0 + ): + yield from self._on_stream_is_done(partition.stream_name()) + yield from self._message_repository.consume_queue() def on_record(self, record: Record) -> Iterable[AirbyteMessage]: """ diff --git a/airbyte_cdk/sources/concurrent_source/concurrent_source.py b/airbyte_cdk/sources/concurrent_source/concurrent_source.py index ffdee2dc1..9ccfc1088 100644 --- a/airbyte_cdk/sources/concurrent_source/concurrent_source.py +++ b/airbyte_cdk/sources/concurrent_source/concurrent_source.py @@ -4,7 +4,7 @@ import concurrent import logging from queue import Queue -from typing import Iterable, Iterator, List +from typing import Iterable, Iterator, List, Optional from airbyte_cdk.models import AirbyteMessage from airbyte_cdk.sources.concurrent_source.concurrent_read_processor import ConcurrentReadProcessor @@ -16,7 +16,7 @@ from airbyte_cdk.sources.message import InMemoryMessageRepository, MessageRepository from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream from airbyte_cdk.sources.streams.concurrent.partition_enqueuer import PartitionEnqueuer -from airbyte_cdk.sources.streams.concurrent.partition_reader import PartitionReader +from airbyte_cdk.sources.streams.concurrent.partition_reader import PartitionLogger, PartitionReader from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.streams.concurrent.partitions.types import ( PartitionCompleteSentinel, @@ -43,6 +43,7 @@ def create( logger: logging.Logger, slice_logger: SliceLogger, message_repository: MessageRepository, + queue: Optional[Queue[QueueItem]] = None, timeout_seconds: int = DEFAULT_TIMEOUT_SECONDS, ) -> "ConcurrentSource": is_single_threaded = initial_number_of_partitions_to_generate == 1 and num_workers == 1 @@ -59,12 +60,13 @@ def create( logger, ) return ConcurrentSource( - threadpool, - logger, - slice_logger, - message_repository, - initial_number_of_partitions_to_generate, - timeout_seconds, + threadpool=threadpool, + logger=logger, + slice_logger=slice_logger, + queue=queue, + message_repository=message_repository, + initial_number_partitions_to_generate=initial_number_of_partitions_to_generate, + timeout_seconds=timeout_seconds, ) def __init__( @@ -72,6 +74,7 @@ def __init__( threadpool: ThreadPoolManager, logger: logging.Logger, slice_logger: SliceLogger = DebugSliceLogger(), + queue: Optional[Queue[QueueItem]] = None, message_repository: MessageRepository = InMemoryMessageRepository(), initial_number_partitions_to_generate: int = 1, timeout_seconds: int = DEFAULT_TIMEOUT_SECONDS, @@ -91,25 +94,28 @@ def __init__( self._initial_number_partitions_to_generate = initial_number_partitions_to_generate self._timeout_seconds = timeout_seconds + # We set a maxsize to for the main thread to process record items when the queue size grows. This assumes that there are less + # threads generating partitions that than are max number of workers. If it weren't the case, we could have threads only generating + # partitions which would fill the queue. This number is arbitrarily set to 10_000 but will probably need to be changed given more + # information and might even need to be configurable depending on the source + self._queue = queue or Queue(maxsize=10_000) + def read( self, streams: List[AbstractStream], ) -> Iterator[AirbyteMessage]: self._logger.info("Starting syncing") - - # We set a maxsize to for the main thread to process record items when the queue size grows. This assumes that there are less - # threads generating partitions that than are max number of workers. If it weren't the case, we could have threads only generating - # partitions which would fill the queue. This number is arbitrarily set to 10_000 but will probably need to be changed given more - # information and might even need to be configurable depending on the source - queue: Queue[QueueItem] = Queue(maxsize=10_000) concurrent_stream_processor = ConcurrentReadProcessor( streams, - PartitionEnqueuer(queue, self._threadpool), + PartitionEnqueuer(self._queue, self._threadpool), self._threadpool, self._logger, self._slice_logger, self._message_repository, - PartitionReader(queue), + PartitionReader( + self._queue, + PartitionLogger(self._slice_logger, self._logger, self._message_repository), + ), ) # Enqueue initial partition generation tasks @@ -117,7 +123,7 @@ def read( # Read from the queue until all partitions were generated and read yield from self._consume_from_queue( - queue, + self._queue, concurrent_stream_processor, ) self._threadpool.check_for_errors_and_shutdown() @@ -141,7 +147,10 @@ def _consume_from_queue( airbyte_message_or_record_or_exception, concurrent_stream_processor, ) - if concurrent_stream_processor.is_done() and queue.empty(): + # In the event that a partition raises an exception, anything remaining in + # the queue will be missed because is_done() can raise an exception and exit + # out of this loop before remaining items are consumed + if queue.empty() and concurrent_stream_processor.is_done(): # all partitions were generated and processed. we're done here break @@ -161,5 +170,7 @@ def _handle_item( yield from concurrent_stream_processor.on_partition_complete_sentinel(queue_item) elif isinstance(queue_item, Record): yield from concurrent_stream_processor.on_record(queue_item) + elif isinstance(queue_item, AirbyteMessage): + yield queue_item else: raise ValueError(f"Unknown queue item type: {type(queue_item)}") diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 7accd1ac6..8cdc69a37 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -3,7 +3,11 @@ # import logging -from typing import Any, Generic, Iterator, List, Mapping, MutableMapping, Optional, Tuple +from dataclasses import dataclass, field +from queue import Queue +from typing import Any, ClassVar, Generic, Iterator, List, Mapping, MutableMapping, Optional, Tuple + +from airbyte_protocol_dataclasses.models import Level from airbyte_cdk.models import ( AirbyteCatalog, @@ -48,6 +52,8 @@ StreamSlicerPartitionGenerator, ) from airbyte_cdk.sources.declarative.types import ConnectionDefinition +from airbyte_cdk.sources.message.concurrent_repository import ConcurrentMessageRepository +from airbyte_cdk.sources.message.repository import InMemoryMessageRepository, MessageRepository from airbyte_cdk.sources.source import TState from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream @@ -58,6 +64,22 @@ from airbyte_cdk.sources.streams.concurrent.cursor import ConcurrentCursor, FinalStateCursor from airbyte_cdk.sources.streams.concurrent.default_stream import DefaultStream from airbyte_cdk.sources.streams.concurrent.helpers import get_primary_key_from_stream +from airbyte_cdk.sources.streams.concurrent.partitions.types import QueueItem + + +@dataclass +class TestLimits: + __test__: ClassVar[bool] = False # Tell Pytest this is not a Pytest class, despite its name + + DEFAULT_MAX_PAGES_PER_SLICE: ClassVar[int] = 5 + DEFAULT_MAX_SLICES: ClassVar[int] = 5 + DEFAULT_MAX_RECORDS: ClassVar[int] = 100 + DEFAULT_MAX_STREAMS: ClassVar[int] = 100 + + max_records: int = field(default=DEFAULT_MAX_RECORDS) + max_pages_per_slice: int = field(default=DEFAULT_MAX_PAGES_PER_SLICE) + max_slices: int = field(default=DEFAULT_MAX_SLICES) + max_streams: int = field(default=DEFAULT_MAX_STREAMS) class ConcurrentDeclarativeSource(ManifestDeclarativeSource, Generic[TState]): @@ -73,7 +95,7 @@ def __init__( source_config: ConnectionDefinition, debug: bool = False, emit_connector_builder_messages: bool = False, - component_factory: Optional[ModelToComponentFactory] = None, + limits: Optional[TestLimits] = None, config_path: Optional[str] = None, **kwargs: Any, ) -> None: @@ -81,17 +103,33 @@ def __init__( # no longer needs to store the original incoming state. But maybe there's an edge case? self._connector_state_manager = ConnectorStateManager(state=state) # type: ignore # state is always in the form of List[AirbyteStateMessage]. The ConnectorStateManager should use generics, but this can be done later + # We set a maxsize to for the main thread to process record items when the queue size grows. This assumes that there are less + # threads generating partitions that than are max number of workers. If it weren't the case, we could have threads only generating + # partitions which would fill the queue. This number is arbitrarily set to 10_000 but will probably need to be changed given more + # information and might even need to be configurable depending on the source + queue: Queue[QueueItem] = Queue(maxsize=10_000) + message_repository = InMemoryMessageRepository( + Level.DEBUG if emit_connector_builder_messages else Level.INFO + ) + # To reduce the complexity of the concurrent framework, we are not enabling RFR with synthetic # cursors. We do this by no longer automatically instantiating RFR cursors when converting # the declarative models into runtime components. Concurrent sources will continue to checkpoint # incremental streams running in full refresh. - component_factory = component_factory or ModelToComponentFactory( + component_factory = ModelToComponentFactory( emit_connector_builder_messages=emit_connector_builder_messages, disable_resumable_full_refresh=True, + message_repository=ConcurrentMessageRepository(queue, message_repository), connector_state_manager=self._connector_state_manager, max_concurrent_async_job_count=source_config.get("max_concurrent_async_job_count"), + limit_pages_fetched_per_slice=limits.max_pages_per_slice if limits else None, + limit_slices_fetched=limits.max_slices if limits else None, + disable_retries=True if limits else False, + disable_cache=True if limits else False, ) + self._limits = limits + super().__init__( source_config=source_config, config=config, @@ -126,6 +164,7 @@ def __init__( initial_number_of_partitions_to_generate=initial_number_of_partitions_to_generate, logger=self.logger, slice_logger=self._slice_logger, + queue=queue, message_repository=self.message_repository, ) @@ -287,6 +326,9 @@ def _group_streams( self.message_repository, ), 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: if ( @@ -318,6 +360,7 @@ def _group_streams( self.message_repository, ), stream_slicer=cursor, + slice_limit=self._limits.max_slices if self._limits else None, ) concurrent_streams.append( @@ -349,6 +392,9 @@ def _group_streams( self.message_repository, ), 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 ) final_state_cursor = FinalStateCursor( @@ -410,6 +456,7 @@ def _group_streams( self.message_repository, ), perpartition_cursor, + slice_limit=self._limits.max_slices if self._limits else None, ) concurrent_streams.append( 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 5d2415525..9f59f855f 100644 --- a/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py +++ b/airbyte_cdk/sources/declarative/parsers/model_to_component_factory.py @@ -622,6 +622,10 @@ SchemaNormalizationModel.Default: TransformConfig.DefaultSchemaNormalization, } +# Ideally this should use the value defined in ConcurrentDeclarativeSource, but +# this would be a circular import +MAX_SLICES = 5 + class ModelToComponentFactory: EPOCH_DATETIME_FORMAT = "%s" diff --git a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py index 94ee03a56..7073e48a6 100644 --- a/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py +++ b/airbyte_cdk/sources/declarative/stream_slicers/declarative_partition_generator.py @@ -1,8 +1,11 @@ -# Copyright (c) 2024 Airbyte, Inc., all rights reserved. +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. -from typing import Any, Iterable, Mapping, Optional +from typing import Any, Iterable, Mapping, Optional, cast from airbyte_cdk.sources.declarative.retrievers import Retriever +from airbyte_cdk.sources.declarative.stream_slicers.stream_slicer_test_read_decorator import ( + StreamSlicerTestReadDecorator, +) from airbyte_cdk.sources.message import MessageRepository from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator @@ -83,10 +86,23 @@ def __hash__(self) -> int: class StreamSlicerPartitionGenerator(PartitionGenerator): def __init__( - self, partition_factory: DeclarativePartitionFactory, stream_slicer: StreamSlicer + self, + partition_factory: DeclarativePartitionFactory, + stream_slicer: StreamSlicer, + slice_limit: Optional[int] = None, ) -> None: self._partition_factory = partition_factory - self._stream_slicer = stream_slicer + + if slice_limit: + self._stream_slicer = cast( + StreamSlicer, + StreamSlicerTestReadDecorator( + wrapped_slicer=stream_slicer, + maximum_number_of_slices=slice_limit, + ), + ) + else: + self._stream_slicer = stream_slicer def generate(self) -> Iterable[Partition]: for stream_slice in self._stream_slicer.stream_slices(): diff --git a/airbyte_cdk/sources/declarative/stream_slicers/stream_slicer_test_read_decorator.py b/airbyte_cdk/sources/declarative/stream_slicers/stream_slicer_test_read_decorator.py index 323c89196..d261c27e8 100644 --- a/airbyte_cdk/sources/declarative/stream_slicers/stream_slicer_test_read_decorator.py +++ b/airbyte_cdk/sources/declarative/stream_slicers/stream_slicer_test_read_decorator.py @@ -4,10 +4,10 @@ from dataclasses import dataclass from itertools import islice -from typing import Any, Iterable, Mapping, Optional, Union +from typing import Any, Iterable from airbyte_cdk.sources.streams.concurrent.partitions.stream_slicer import StreamSlicer -from airbyte_cdk.sources.types import StreamSlice, StreamState +from airbyte_cdk.sources.types import StreamSlice @dataclass diff --git a/airbyte_cdk/sources/message/concurrent_repository.py b/airbyte_cdk/sources/message/concurrent_repository.py new file mode 100644 index 000000000..947ee4c46 --- /dev/null +++ b/airbyte_cdk/sources/message/concurrent_repository.py @@ -0,0 +1,43 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. + +from queue import Queue +from typing import Callable, Iterable + +from airbyte_cdk.models import AirbyteMessage, Level +from airbyte_cdk.sources.message.repository import LogMessage, MessageRepository +from airbyte_cdk.sources.streams.concurrent.partitions.types import QueueItem + + +class ConcurrentMessageRepository(MessageRepository): + """ + Message repository that immediately loads messages onto the queue processed on the + main thread. This ensures that messages are processed in the correct order they are + received. The InMemoryMessageRepository implementation does not have guaranteed + ordering since whether to process the main thread vs. partitions is non-deterministic + and there can be a lag between reading the main-thread and consuming messages on the + MessageRepository. + + This is particularly important for the connector builder which relies on grouping + of messages to organize request/response, pages, and partitions. + """ + + def __init__(self, queue: Queue[QueueItem], message_repository: MessageRepository): + self._queue = queue + self._decorated_message_repository = message_repository + + def emit_message(self, message: AirbyteMessage) -> None: + self._decorated_message_repository.emit_message(message) + for message in self._decorated_message_repository.consume_queue(): + self._queue.put(message) + + def log_message(self, level: Level, message_provider: Callable[[], LogMessage]) -> None: + self._decorated_message_repository.log_message(level, message_provider) + for message in self._decorated_message_repository.consume_queue(): + self._queue.put(message) + + def consume_queue(self) -> Iterable[AirbyteMessage]: + """ + This method shouldn't need to be called because as part of emit_message() we are already + loading messages onto the queue processed on the main thread. + """ + yield from [] diff --git a/airbyte_cdk/sources/streams/concurrent/partition_reader.py b/airbyte_cdk/sources/streams/concurrent/partition_reader.py index 3d23fd9cf..39bb2de2e 100644 --- a/airbyte_cdk/sources/streams/concurrent/partition_reader.py +++ b/airbyte_cdk/sources/streams/concurrent/partition_reader.py @@ -1,14 +1,45 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. + +import logging from queue import Queue +from typing import Optional from airbyte_cdk.sources.concurrent_source.stream_thread_exception import StreamThreadException +from airbyte_cdk.sources.message.repository import MessageRepository +from airbyte_cdk.sources.streams.concurrent.cursor import Cursor from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.streams.concurrent.partitions.types import ( PartitionCompleteSentinel, QueueItem, ) +from airbyte_cdk.sources.utils.slice_logger import SliceLogger + + +# Since moving all the connector builder workflow to the concurrent CDK which required correct ordering +# of grouping log messages onto the main write thread using the ConcurrentMessageRepository, this +# separate flow and class that was used to log slices onto this partition's message_repository +# should just be replaced by emitting messages directly onto the repository instead of an intermediary. +class PartitionLogger: + """ + Helper class that provides a mechanism for passing a log message onto the current + partitions message repository + """ + + def __init__( + self, + slice_logger: SliceLogger, + logger: logging.Logger, + message_repository: MessageRepository, + ): + self._slice_logger = slice_logger + self._logger = logger + self._message_repository = message_repository + + def log(self, partition: Partition) -> None: + if self._slice_logger.should_log_slice_message(self._logger): + self._message_repository.emit_message( + self._slice_logger.create_slice_log_message(partition.to_slice()) + ) class PartitionReader: @@ -18,13 +49,16 @@ class PartitionReader: _IS_SUCCESSFUL = True - def __init__(self, queue: Queue[QueueItem]) -> None: + def __init__( + self, queue: Queue[QueueItem], partition_logger: Optional[PartitionLogger] = None + ) -> None: """ :param queue: The queue to put the records in. """ self._queue = queue + self._partition_logger = partition_logger - def process_partition(self, partition: Partition) -> None: + def process_partition(self, partition: Partition, cursor: Cursor) -> None: """ Process a partition and put the records in the output queue. When all the partitions are added to the queue, a sentinel is added to the queue to indicate that all the partitions have been generated. @@ -37,8 +71,13 @@ def process_partition(self, partition: Partition) -> None: :return: None """ try: + if self._partition_logger: + self._partition_logger.log(partition) + for record in partition.read(): self._queue.put(record) + cursor.observe(record) + cursor.close_partition(partition) self._queue.put(PartitionCompleteSentinel(partition, self._IS_SUCCESSFUL)) except Exception as e: self._queue.put(StreamThreadException(e, partition.stream_name())) diff --git a/airbyte_cdk/sources/streams/concurrent/partitions/types.py b/airbyte_cdk/sources/streams/concurrent/partitions/types.py index 77644c6b9..3ae63c242 100644 --- a/airbyte_cdk/sources/streams/concurrent/partitions/types.py +++ b/airbyte_cdk/sources/streams/concurrent/partitions/types.py @@ -4,6 +4,7 @@ from typing import Any, Union +from airbyte_cdk.models import AirbyteMessage from airbyte_cdk.sources.concurrent_source.partition_generation_completed_sentinel import ( PartitionGenerationCompletedSentinel, ) @@ -34,5 +35,10 @@ def __eq__(self, other: Any) -> bool: Typedef representing the items that can be added to the ThreadBasedConcurrentStream """ QueueItem = Union[ - Record, Partition, PartitionCompleteSentinel, PartitionGenerationCompletedSentinel, Exception + Record, + Partition, + PartitionCompleteSentinel, + PartitionGenerationCompletedSentinel, + Exception, + AirbyteMessage, ] diff --git a/airbyte_cdk/sources/utils/slice_logger.py b/airbyte_cdk/sources/utils/slice_logger.py index ee802a7a6..4b29f3e0d 100644 --- a/airbyte_cdk/sources/utils/slice_logger.py +++ b/airbyte_cdk/sources/utils/slice_logger.py @@ -11,6 +11,10 @@ from airbyte_cdk.models import Type as MessageType +# Once everything runs on the concurrent CDK and we've cleaned up the legacy flows, we should try to remove +# this class and write messages directly to the message_repository instead of through the logger because for +# cases like the connector builder where ordering of messages is important, using the logger can cause +# messages to be grouped out of order. Alas work for a different day. class SliceLogger(ABC): """ SliceLogger is an interface that allows us to log slices of data in a uniform way. diff --git a/unit_tests/connector_builder/test_connector_builder_handler.py b/unit_tests/connector_builder/test_connector_builder_handler.py index 2587fb95a..993b42e8e 100644 --- a/unit_tests/connector_builder/test_connector_builder_handler.py +++ b/unit_tests/connector_builder/test_connector_builder_handler.py @@ -17,9 +17,6 @@ from airbyte_cdk import connector_builder from airbyte_cdk.connector_builder.connector_builder_handler import ( - DEFAULT_MAXIMUM_NUMBER_OF_PAGES_PER_SLICE, - DEFAULT_MAXIMUM_NUMBER_OF_SLICES, - DEFAULT_MAXIMUM_RECORDS, TestLimits, create_source, get_limits, @@ -56,8 +53,11 @@ Type, ) from airbyte_cdk.models import Type as MessageType +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, + TestLimits, +) from airbyte_cdk.sources.declarative.declarative_stream import DeclarativeStream -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource from airbyte_cdk.sources.declarative.retrievers.simple_retriever import SimpleRetriever from airbyte_cdk.sources.declarative.stream_slicers import StreamSlicerTestReadDecorator from airbyte_cdk.test.mock_http import HttpMocker, HttpRequest, HttpResponse @@ -530,7 +530,9 @@ def test_resolve_manifest(valid_resolve_manifest_config_file): config = copy.deepcopy(RESOLVE_MANIFEST_CONFIG) command = "resolve_manifest" config["__command"] = command - source = ManifestDeclarativeSource(source_config=MANIFEST) + source = ConcurrentDeclarativeSource( + catalog=None, config=config, state=None, source_config=MANIFEST + ) limits = TestLimits() resolved_manifest = handle_connector_builder_request( source, command, config, create_configured_catalog("dummy_stream"), _A_STATE, limits @@ -679,19 +681,21 @@ def test_resolve_manifest(valid_resolve_manifest_config_file): def test_resolve_manifest_error_returns_error_response(): - class MockManifestDeclarativeSource: + class MockConcurrentDeclarativeSource: @property def resolved_manifest(self): raise ValueError - source = MockManifestDeclarativeSource() + source = MockConcurrentDeclarativeSource() response = resolve_manifest(source) assert "Error resolving manifest" in response.trace.error.message def test_read(): config = TEST_READ_CONFIG - source = ManifestDeclarativeSource(source_config=MANIFEST) + source = ConcurrentDeclarativeSource( + catalog=None, config=config, state=None, source_config=MANIFEST + ) real_record = AirbyteRecordMessage( data={"id": "1234", "key": "value"}, emitted_at=1, stream=_stream_name @@ -780,7 +784,9 @@ def test_config_update() -> None: "client_secret": "a client secret", "refresh_token": "a refresh token", } - source = ManifestDeclarativeSource(source_config=manifest) + source = ConcurrentDeclarativeSource( + catalog=None, config=config, state=None, source_config=manifest + ) refresh_request_response = { "access_token": "an updated access token", @@ -817,7 +823,7 @@ def cursor_field(self): def name(self): return _stream_name - class MockManifestDeclarativeSource: + class MockConcurrentDeclarativeSource: def streams(self, config): return [MockDeclarativeStream()] @@ -839,7 +845,7 @@ def check_config_against_spec(self) -> Literal[False]: stack_trace = "a stack trace" mock_from_exception.return_value = stack_trace - source = MockManifestDeclarativeSource() + source = MockConcurrentDeclarativeSource() limits = TestLimits() response = read_stream( source, @@ -881,19 +887,22 @@ def test_handle_429_response(): config = TEST_READ_CONFIG limits = TestLimits() - source = create_source(config, limits) + catalog = ConfiguredAirbyteCatalogSerializer.load(CONFIGURED_CATALOG) + source = create_source(config=config, limits=limits, catalog=catalog, state=None) with patch("requests.Session.send", return_value=response) as mock_send: response = handle_connector_builder_request( source, "test_read", config, - ConfiguredAirbyteCatalogSerializer.load(CONFIGURED_CATALOG), + catalog, _A_PER_PARTITION_STATE, limits, ) - mock_send.assert_called_once() + # The test read will attempt a read for 5 partitions, and attempt 1 request + # each time that will not be retried + assert mock_send.call_count == 5 @pytest.mark.parametrize( @@ -945,7 +954,7 @@ def test_invalid_config_command(invalid_config_file, dummy_catalog): @pytest.fixture def manifest_declarative_source(): - return mock.Mock(spec=ManifestDeclarativeSource, autospec=True) + return mock.Mock(spec=ConcurrentDeclarativeSource, autospec=True) def create_mock_retriever(name, url_base, path): @@ -970,16 +979,16 @@ def create_mock_declarative_stream(http_stream): ( "test_no_test_read_config", {}, - DEFAULT_MAXIMUM_RECORDS, - DEFAULT_MAXIMUM_NUMBER_OF_SLICES, - DEFAULT_MAXIMUM_NUMBER_OF_PAGES_PER_SLICE, + TestLimits.DEFAULT_MAX_RECORDS, + TestLimits.DEFAULT_MAX_SLICES, + TestLimits.DEFAULT_MAX_PAGES_PER_SLICE, ), ( "test_no_values_set", {"__test_read_config": {}}, - DEFAULT_MAXIMUM_RECORDS, - DEFAULT_MAXIMUM_NUMBER_OF_SLICES, - DEFAULT_MAXIMUM_NUMBER_OF_PAGES_PER_SLICE, + TestLimits.DEFAULT_MAX_RECORDS, + TestLimits.DEFAULT_MAX_SLICES, + TestLimits.DEFAULT_MAX_PAGES_PER_SLICE, ), ( "test_values_are_set", @@ -1007,9 +1016,9 @@ def test_create_source(): config = {"__injected_declarative_manifest": MANIFEST} - source = create_source(config, limits) + source = create_source(config=config, limits=limits, catalog=None, state=None) - assert isinstance(source, ManifestDeclarativeSource) + assert isinstance(source, ConcurrentDeclarativeSource) assert source._constructor._limit_pages_fetched_per_slice == limits.max_pages_per_slice assert source._constructor._limit_slices_fetched == limits.max_slices assert source._constructor._disable_cache @@ -1101,7 +1110,7 @@ def test_read_source(mock_http_stream): config = {"__injected_declarative_manifest": MANIFEST} - source = create_source(config, limits) + source = create_source(config=config, limits=limits, catalog=catalog, state=None) output_data = read_stream(source, config, catalog, _A_PER_PARTITION_STATE, limits).record.data slices = output_data["slices"] @@ -1149,7 +1158,7 @@ def test_read_source_single_page_single_slice(mock_http_stream): config = {"__injected_declarative_manifest": MANIFEST} - source = create_source(config, limits) + source = create_source(config=config, limits=limits, catalog=catalog, state=None) output_data = read_stream(source, config, catalog, _A_PER_PARTITION_STATE, limits).record.data slices = output_data["slices"] @@ -1236,7 +1245,7 @@ def test_handle_read_external_requests(deployment_mode, url_base, expected_error test_manifest["streams"][0]["$parameters"]["url_base"] = url_base config = {"__injected_declarative_manifest": test_manifest} - source = create_source(config, limits) + source = create_source(config=config, limits=limits, catalog=catalog, state=None) with mock.patch.dict(os.environ, {"DEPLOYMENT_MODE": deployment_mode}, clear=False): output_data = read_stream( @@ -1266,13 +1275,13 @@ def test_handle_read_external_requests(deployment_mode, url_base, expected_error pytest.param( "CLOUD", "https://10.0.27.27/tokens/bearer", - "AirbyteTracedException", + "StreamThreadException", id="test_cloud_read_with_private_endpoint", ), pytest.param( "CLOUD", "http://unsecured.protocol/tokens/bearer", - "InvalidSchema", + "StreamThreadException", id="test_cloud_read_with_unsecured_endpoint", ), pytest.param( @@ -1332,7 +1341,7 @@ def test_handle_read_external_oauth_request(deployment_mode, token_url, expected ) config = {"__injected_declarative_manifest": test_manifest} - source = create_source(config, limits) + source = create_source(config=config, limits=limits, catalog=catalog, state=None) with mock.patch.dict(os.environ, {"DEPLOYMENT_MODE": deployment_mode}, clear=False): output_data = read_stream( @@ -1389,7 +1398,9 @@ def test_read_stream_exception_with_secrets(): def test_full_resolve_manifest(valid_resolve_manifest_config_file): config = copy.deepcopy(RESOLVE_DYNAMIC_STREAM_MANIFEST_CONFIG) command = config["__command"] - source = ManifestDeclarativeSource(source_config=DYNAMIC_STREAM_MANIFEST) + source = ConcurrentDeclarativeSource( + catalog=None, config=config, state=None, source_config=DYNAMIC_STREAM_MANIFEST + ) limits = TestLimits(max_streams=2) with HttpMocker() as http_mocker: http_mocker.get( diff --git a/unit_tests/sources/declarative/schema/test_dynamic_schema_loader.py b/unit_tests/sources/declarative/schema/test_dynamic_schema_loader.py index 97f89879c..20147465f 100644 --- a/unit_tests/sources/declarative/schema/test_dynamic_schema_loader.py +++ b/unit_tests/sources/declarative/schema/test_dynamic_schema_loader.py @@ -10,9 +10,7 @@ from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( ConcurrentDeclarativeSource, -) -from airbyte_cdk.sources.declarative.parsers.model_to_component_factory import ( - ModelToComponentFactory, + TestLimits, ) from airbyte_cdk.sources.declarative.schema import DynamicSchemaLoader, SchemaTypeIdentifier from airbyte_cdk.test.mock_http import HttpMocker, HttpRequest, HttpResponse @@ -353,14 +351,13 @@ def test_dynamic_schema_loader_with_type_conditions(): }, }, } + source = ConcurrentDeclarativeSource( source_config=_MANIFEST_WITH_TYPE_CONDITIONS, config=_CONFIG, catalog=None, state=None, - component_factory=ModelToComponentFactory( - disable_cache=True - ), # Avoid caching on the HttpClient which could result in caching the requests/responses of other tests + limits=TestLimits(), # Avoid caching on the HttpClient which could result in caching the requests/responses of other tests ) with HttpMocker() as http_mocker: http_mocker.get( diff --git a/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py b/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py index 50695ba1e..75b52f6b2 100644 --- a/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py +++ b/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py @@ -50,7 +50,10 @@ def __init__( self._message_repository = InMemoryMessageRepository() threadpool_manager = ThreadPoolManager(threadpool, streams[0].logger) concurrent_source = ConcurrentSource( - threadpool_manager, streams[0].logger, NeverLogSliceLogger(), self._message_repository + threadpool=threadpool_manager, + logger=streams[0].logger, + slice_logger=NeverLogSliceLogger(), + message_repository=self._message_repository, ) super().__init__(concurrent_source) self._streams = streams diff --git a/unit_tests/sources/streams/concurrent/test_concurrent_read_processor.py b/unit_tests/sources/streams/concurrent/test_concurrent_read_processor.py index d6ea64583..a681f75eb 100644 --- a/unit_tests/sources/streams/concurrent/test_concurrent_read_processor.py +++ b/unit_tests/sources/streams/concurrent/test_concurrent_read_processor.py @@ -176,10 +176,12 @@ def test_handle_partition(self): self._partition_reader, ) + expected_cursor = handler._stream_name_to_instance[_ANOTHER_STREAM_NAME].cursor + handler.on_partition(self._a_closed_partition) self._thread_pool_manager.submit.assert_called_with( - self._partition_reader.process_partition, self._a_closed_partition + self._partition_reader.process_partition, self._a_closed_partition, expected_cursor ) assert ( self._a_closed_partition in handler._streams_to_running_partitions[_ANOTHER_STREAM_NAME] @@ -201,10 +203,12 @@ def test_handle_partition_emits_log_message_if_it_should_be_logged(self): self._partition_reader, ) + expected_cursor = handler._stream_name_to_instance[_STREAM_NAME].cursor + handler.on_partition(self._an_open_partition) self._thread_pool_manager.submit.assert_called_with( - self._partition_reader.process_partition, self._an_open_partition + self._partition_reader.process_partition, self._an_open_partition, expected_cursor ) self._message_repository.emit_message.assert_called_with(self._log_message) @@ -253,8 +257,6 @@ def test_handle_on_partition_complete_sentinel_with_messages_from_repository(sel ] assert messages == expected_messages - self._stream.cursor.close_partition.assert_called_once() - @freezegun.freeze_time("2020-01-01T00:00:00") def test_handle_on_partition_complete_sentinel_yields_status_message_if_the_stream_is_done( self, @@ -302,55 +304,6 @@ def test_handle_on_partition_complete_sentinel_yields_status_message_if_the_stre ) ] assert messages == expected_messages - self._another_stream.cursor.close_partition.assert_called_once() - - @freezegun.freeze_time("2020-01-01T00:00:00") - def test_given_exception_on_partition_complete_sentinel_then_yield_error_trace_message_and_stream_is_incomplete( - self, - ) -> None: - self._a_closed_partition.stream_name.return_value = self._stream.name - self._stream.cursor.close_partition.side_effect = ValueError - - handler = ConcurrentReadProcessor( - [self._stream], - self._partition_enqueuer, - self._thread_pool_manager, - self._logger, - self._slice_logger, - self._message_repository, - self._partition_reader, - ) - handler.start_next_partition_generator() - handler.on_partition(self._a_closed_partition) - list( - handler.on_partition_generation_completed( - PartitionGenerationCompletedSentinel(self._stream) - ) - ) - messages = list( - handler.on_partition_complete_sentinel( - PartitionCompleteSentinel(self._a_closed_partition) - ) - ) - - expected_status_message = AirbyteMessage( - type=MessageType.TRACE, - trace=AirbyteTraceMessage( - type=TraceType.STREAM_STATUS, - stream_status=AirbyteStreamStatusTraceMessage( - stream_descriptor=StreamDescriptor( - name=self._stream.name, - ), - status=AirbyteStreamStatus.INCOMPLETE, - ), - emitted_at=1577836800000.0, - ), - ) - assert list(map(lambda message: message.trace.type, messages)) == [ - TraceType.ERROR, - TraceType.STREAM_STATUS, - ] - assert messages[1] == expected_status_message @freezegun.freeze_time("2020-01-01T00:00:00") def test_handle_on_partition_complete_sentinel_yields_no_status_message_if_the_stream_is_not_done( @@ -379,7 +332,6 @@ def test_handle_on_partition_complete_sentinel_yields_no_status_message_if_the_s expected_messages = [] assert messages == expected_messages - self._stream.cursor.close_partition.assert_called_once() @freezegun.freeze_time("2020-01-01T00:00:00") def test_on_record_no_status_message_no_repository_messge(self): diff --git a/unit_tests/sources/streams/concurrent/test_partition_reader.py b/unit_tests/sources/streams/concurrent/test_partition_reader.py index 1910e034d..a41750772 100644 --- a/unit_tests/sources/streams/concurrent/test_partition_reader.py +++ b/unit_tests/sources/streams/concurrent/test_partition_reader.py @@ -1,6 +1,5 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. + import unittest from queue import Queue from typing import Callable, Iterable, List @@ -8,7 +7,9 @@ import pytest +from airbyte_cdk import InMemoryMessageRepository from airbyte_cdk.sources.concurrent_source.stream_thread_exception import StreamThreadException +from airbyte_cdk.sources.streams.concurrent.cursor import FinalStateCursor from airbyte_cdk.sources.streams.concurrent.partition_reader import PartitionReader from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.streams.concurrent.partitions.types import ( @@ -26,10 +27,15 @@ class PartitionReaderTest(unittest.TestCase): def setUp(self) -> None: self._queue: Queue[QueueItem] = Queue() - self._partition_reader = PartitionReader(self._queue) + self._partition_reader = PartitionReader(self._queue, None) def test_given_no_records_when_process_partition_then_only_emit_sentinel(self): - self._partition_reader.process_partition(self._a_partition([])) + cursor = FinalStateCursor( + stream_name="test", + stream_namespace=None, + message_repository=InMemoryMessageRepository(), + ) + self._partition_reader.process_partition(self._a_partition([]), cursor) while queue_item := self._queue.get(): if not isinstance(queue_item, PartitionCompleteSentinel): @@ -40,19 +46,24 @@ def test_given_read_partition_successful_when_process_partition_then_queue_recor self, ): partition = self._a_partition(_RECORDS) - self._partition_reader.process_partition(partition) + cursor = Mock() + self._partition_reader.process_partition(partition, cursor) queue_content = self._consume_queue() assert queue_content == _RECORDS + [PartitionCompleteSentinel(partition)] - def test_given_exception_when_process_partition_then_queue_records_and_exception_and_sentinel( + cursor.observe.assert_called() + cursor.close_partition.assert_called_once() + + def test_given_exception_from_read_when_process_partition_then_queue_records_and_exception_and_sentinel( self, ): partition = Mock() + cursor = Mock() exception = ValueError() partition.read.side_effect = self._read_with_exception(_RECORDS, exception) - self._partition_reader.process_partition(partition) + self._partition_reader.process_partition(partition, cursor) queue_content = self._consume_queue() @@ -61,6 +72,23 @@ def test_given_exception_when_process_partition_then_queue_records_and_exception PartitionCompleteSentinel(partition), ] + def test_given_exception_from_close_slice_when_process_partition_then_queue_records_and_exception_and_sentinel( + self, + ): + partition = self._a_partition(_RECORDS) + cursor = Mock() + exception = ValueError() + cursor.close_partition.side_effect = self._close_partition_with_exception(exception) + self._partition_reader.process_partition(partition, cursor) + + queue_content = self._consume_queue() + + # 4 total messages in queue. 2 records, 1 thread exception, 1 partition sentinel value + assert len(queue_content) == 4 + assert queue_content[:2] == _RECORDS + assert isinstance(queue_content[2], StreamThreadException) + assert queue_content[3] == PartitionCompleteSentinel(partition) + def _a_partition(self, records: List[Record]) -> Partition: partition = Mock(spec=Partition) partition.read.return_value = iter(records) @@ -76,6 +104,13 @@ def mocked_function() -> Iterable[Record]: return mocked_function + @staticmethod + def _close_partition_with_exception(exception: Exception) -> Callable[[Partition], None]: + def mocked_function(partition: Partition) -> None: + raise exception + + return mocked_function + def _consume_queue(self): queue_content = [] while queue_item := self._queue.get():