diff --git a/airbyte_cdk/__init__.py b/airbyte_cdk/__init__.py index 50fe2e21a..5268a9654 100644 --- a/airbyte_cdk/__init__.py +++ b/airbyte_cdk/__init__.py @@ -107,7 +107,6 @@ from .sources.declarative.extractors.record_filter import RecordFilter from .sources.declarative.incremental import DatetimeBasedCursor from .sources.declarative.interpolation import InterpolatedBoolean, InterpolatedString -from .sources.declarative.manifest_declarative_source import ManifestDeclarativeSource from .sources.declarative.migrations.legacy_to_per_partition_state_migration import ( LegacyToPerPartitionStateMigration, ) @@ -253,7 +252,6 @@ "JsonDecoder", "JsonFileSchemaLoader", "LegacyToPerPartitionStateMigration", - "ManifestDeclarativeSource", "MinMaxDatetime", "NoAuth", "OffsetIncrement", diff --git a/airbyte_cdk/connector_builder/connector_builder_handler.py b/airbyte_cdk/connector_builder/connector_builder_handler.py index a7d2163a9..0d3e2052b 100644 --- a/airbyte_cdk/connector_builder/connector_builder_handler.py +++ b/airbyte_cdk/connector_builder/connector_builder_handler.py @@ -19,8 +19,6 @@ ConcurrentDeclarativeSource, TestLimits, ) -from airbyte_cdk.sources.declarative.declarative_source import DeclarativeSource -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource 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 @@ -90,7 +88,7 @@ def create_source( def read_stream( - source: DeclarativeSource, + source: ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]], config: Mapping[str, Any], configured_catalog: ConfiguredAirbyteCatalog, state: List[AirbyteStateMessage], @@ -128,7 +126,9 @@ def read_stream( return error.as_airbyte_message() -def resolve_manifest(source: ManifestDeclarativeSource) -> AirbyteMessage: +def resolve_manifest( + source: ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]], +) -> AirbyteMessage: try: return AirbyteMessage( type=Type.RECORD, @@ -145,7 +145,9 @@ def resolve_manifest(source: ManifestDeclarativeSource) -> AirbyteMessage: return error.as_airbyte_message() -def full_resolve_manifest(source: ManifestDeclarativeSource, limits: TestLimits) -> AirbyteMessage: +def full_resolve_manifest( + source: ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]], limits: TestLimits +) -> AirbyteMessage: try: manifest = {**source.resolved_manifest} streams = manifest.get("streams", []) diff --git a/airbyte_cdk/connector_builder/main.py b/airbyte_cdk/connector_builder/main.py index 22be81c82..207831c3c 100644 --- a/airbyte_cdk/connector_builder/main.py +++ b/airbyte_cdk/connector_builder/main.py @@ -25,7 +25,9 @@ ConfiguredAirbyteCatalog, ConfiguredAirbyteCatalogSerializer, ) -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) from airbyte_cdk.sources.source import Source from airbyte_cdk.utils.traced_exception import AirbyteTracedException @@ -68,7 +70,7 @@ def get_config_and_catalog_from_args( def handle_connector_builder_request( - source: ManifestDeclarativeSource, + source: ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]], command: str, config: Mapping[str, Any], catalog: Optional[ConfiguredAirbyteCatalog], diff --git a/airbyte_cdk/connector_builder/test_reader/reader.py b/airbyte_cdk/connector_builder/test_reader/reader.py index e7399f3f6..aab700951 100644 --- a/airbyte_cdk/connector_builder/test_reader/reader.py +++ b/airbyte_cdk/connector_builder/test_reader/reader.py @@ -23,7 +23,9 @@ ConfiguredAirbyteCatalog, TraceType, ) -from airbyte_cdk.sources.declarative.declarative_source import DeclarativeSource +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) from airbyte_cdk.utils import AirbyteTracedException from airbyte_cdk.utils.datetime_format_inferrer import DatetimeFormatInferrer from airbyte_cdk.utils.schema_inferrer import ( @@ -55,7 +57,7 @@ class TestReader: that contains slices of data, log messages, auxiliary requests, and any inferred schema or datetime formats. Parameters: - source (DeclarativeSource): The data source to read from. + source (ConcurrentDeclarativeSource): The data source to read from. config (Mapping[str, Any]): Configuration parameters for the source. configured_catalog (ConfiguredAirbyteCatalog): Catalog containing stream configuration. state (List[AirbyteStateMessage]): Current state information for the read. @@ -83,7 +85,7 @@ def __init__( def run_test_read( self, - source: DeclarativeSource, + source: ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]], config: Mapping[str, Any], configured_catalog: ConfiguredAirbyteCatalog, stream_name: str, @@ -94,7 +96,7 @@ def run_test_read( Run a test read for the connector by reading from a single stream and inferring schema and datetime formats. Parameters: - source (DeclarativeSource): The source instance providing the streams. + source (ConcurrentDeclarativeSource): The source instance providing the streams. config (Mapping[str, Any]): The configuration settings to use for reading. configured_catalog (ConfiguredAirbyteCatalog): The catalog specifying the stream configuration. state (List[AirbyteStateMessage]): A list of state messages to resume the read. @@ -126,7 +128,7 @@ def run_test_read( if stream else None, self._cursor_field_to_nested_and_composite_field(stream.cursor_field) - if stream + if stream and stream.cursor_field else None, ) datetime_format_inferrer = DatetimeFormatInferrer() @@ -381,13 +383,13 @@ def _get_latest_config_update( def _read_stream( self, - source: DeclarativeSource, + source: ConcurrentDeclarativeSource[Optional[List[AirbyteStateMessage]]], config: Mapping[str, Any], configured_catalog: ConfiguredAirbyteCatalog, state: List[AirbyteStateMessage], ) -> Iterator[AirbyteMessage]: """ - Reads messages from the given DeclarativeSource using an AirbyteEntrypoint. + Reads messages from the given ConcurrentDeclarativeSource using an AirbyteEntrypoint. This method attempts to yield messages from the source's read generator. If the generator raises an AirbyteTracedException, it checks whether the exception message indicates a non-actionable @@ -396,7 +398,7 @@ def _read_stream( wrapped into an AirbyteTracedException, and yielded as an AirbyteMessage. Parameters: - source (DeclarativeSource): The source object that provides data reading logic. + source (ConcurrentDeclarativeSource): The source object that provides data reading logic. config (Mapping[str, Any]): The configuration dictionary for the source. configured_catalog (ConfiguredAirbyteCatalog): The catalog defining the streams and their configurations. state (List[AirbyteStateMessage]): A list representing the current state for incremental sync. diff --git a/airbyte_cdk/legacy/__init__.py b/airbyte_cdk/legacy/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/airbyte_cdk/legacy/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/airbyte_cdk/legacy/sources/__init__.py b/airbyte_cdk/legacy/sources/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/airbyte_cdk/legacy/sources/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/airbyte_cdk/legacy/sources/declarative/__init__.py b/airbyte_cdk/legacy/sources/declarative/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/airbyte_cdk/legacy/sources/declarative/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/airbyte_cdk/sources/declarative/declarative_source.py b/airbyte_cdk/legacy/sources/declarative/declarative_source.py similarity index 100% rename from airbyte_cdk/sources/declarative/declarative_source.py rename to airbyte_cdk/legacy/sources/declarative/declarative_source.py diff --git a/airbyte_cdk/legacy/sources/declarative/incremental/__init__.py b/airbyte_cdk/legacy/sources/declarative/incremental/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/airbyte_cdk/legacy/sources/declarative/incremental/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py b/airbyte_cdk/legacy/sources/declarative/incremental/per_partition_cursor.py similarity index 100% rename from airbyte_cdk/sources/declarative/incremental/per_partition_cursor.py rename to airbyte_cdk/legacy/sources/declarative/incremental/per_partition_cursor.py diff --git a/airbyte_cdk/sources/declarative/manifest_declarative_source.py b/airbyte_cdk/legacy/sources/declarative/manifest_declarative_source.py similarity index 99% rename from airbyte_cdk/sources/declarative/manifest_declarative_source.py rename to airbyte_cdk/legacy/sources/declarative/manifest_declarative_source.py index b1736f371..0f452f97e 100644 --- a/airbyte_cdk/sources/declarative/manifest_declarative_source.py +++ b/airbyte_cdk/legacy/sources/declarative/manifest_declarative_source.py @@ -20,6 +20,7 @@ from airbyte_cdk.connector_builder.models import ( LogMessage as ConnectorBuilderLogMessage, ) +from airbyte_cdk.legacy.sources.declarative.declarative_source import DeclarativeSource from airbyte_cdk.manifest_migrations.migration_handler import ( ManifestMigrationHandler, ) @@ -34,7 +35,6 @@ from airbyte_cdk.models.airbyte_protocol_serializers import AirbyteMessageSerializer from airbyte_cdk.sources.declarative.checks import COMPONENTS_CHECKER_TYPE_MAPPING from airbyte_cdk.sources.declarative.checks.connection_checker import ConnectionChecker -from airbyte_cdk.sources.declarative.declarative_source import DeclarativeSource from airbyte_cdk.sources.declarative.interpolation import InterpolatedBoolean from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( ConditionalStreams as ConditionalStreamsModel, diff --git a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py index 9a651514b..16ff94abf 100644 --- a/airbyte_cdk/sources/declarative/concurrent_declarative_source.py +++ b/airbyte_cdk/sources/declarative/concurrent_declarative_source.py @@ -1,33 +1,55 @@ -# -# Copyright (c) 2024 Airbyte, Inc., all rights reserved. -# +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. +import json import logging +import pkgutil +from copy import deepcopy from dataclasses import dataclass, field from queue import Queue +from types import ModuleType from typing import ( Any, ClassVar, + Dict, Generic, Iterator, List, Mapping, MutableMapping, Optional, + Set, Tuple, Union, ) +import orjson +import yaml from airbyte_protocol_dataclasses.models import Level +from jsonschema.exceptions import ValidationError +from jsonschema.validators import validate +from airbyte_cdk.config_observation import create_connector_config_control_message +from airbyte_cdk.connector_builder.models import ( + LogMessage as ConnectorBuilderLogMessage, +) +from airbyte_cdk.manifest_migrations.migration_handler import ( + ManifestMigrationHandler, +) from airbyte_cdk.models import ( AirbyteCatalog, + AirbyteConnectionStatus, AirbyteMessage, AirbyteStateMessage, ConfiguredAirbyteCatalog, + ConnectorSpecification, + FailureType, ) +from airbyte_cdk.models.airbyte_protocol_serializers import AirbyteMessageSerializer +from airbyte_cdk.sources.abstract_source import AbstractSource from airbyte_cdk.sources.concurrent_source.concurrent_source import ConcurrentSource from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.declarative.checks import COMPONENTS_CHECKER_TYPE_MAPPING +from airbyte_cdk.sources.declarative.checks.connection_checker import ConnectionChecker from airbyte_cdk.sources.declarative.concurrency_level import ConcurrencyLevel from airbyte_cdk.sources.declarative.declarative_stream import DeclarativeStream from airbyte_cdk.sources.declarative.incremental import ( @@ -38,26 +60,50 @@ from airbyte_cdk.sources.declarative.incremental.per_partition_with_global import ( PerPartitionWithGlobalCursor, ) -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource +from airbyte_cdk.sources.declarative.interpolation import InterpolatedBoolean +from airbyte_cdk.sources.declarative.models import FileUploader from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( ConcurrencyLevel as ConcurrencyLevelModel, ) from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( DatetimeBasedCursor as DatetimeBasedCursorModel, ) +from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( + DeclarativeStream as DeclarativeStreamModel, +) from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( IncrementingCountCursor as IncrementingCountCursorModel, ) +from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( + Spec as SpecModel, +) +from airbyte_cdk.sources.declarative.models.declarative_component_schema import ( + StateDelegatingStream as StateDelegatingStreamModel, +) +from airbyte_cdk.sources.declarative.parsers.custom_code_compiler import ( + get_registered_components_module, +) +from airbyte_cdk.sources.declarative.parsers.manifest_component_transformer import ( + ManifestComponentTransformer, +) +from airbyte_cdk.sources.declarative.parsers.manifest_normalizer import ( + ManifestNormalizer, +) +from airbyte_cdk.sources.declarative.parsers.manifest_reference_resolver import ( + ManifestReferenceResolver, +) from airbyte_cdk.sources.declarative.parsers.model_to_component_factory import ( ModelToComponentFactory, ) from airbyte_cdk.sources.declarative.partition_routers import AsyncJobPartitionRouter +from airbyte_cdk.sources.declarative.resolvers import COMPONENTS_RESOLVER_TYPE_MAPPING from airbyte_cdk.sources.declarative.retrievers import AsyncRetriever, Retriever, SimpleRetriever +from airbyte_cdk.sources.declarative.spec.spec import Spec from airbyte_cdk.sources.declarative.stream_slicers.declarative_partition_generator import ( DeclarativePartitionFactory, StreamSlicerPartitionGenerator, ) -from airbyte_cdk.sources.declarative.types import ConnectionDefinition +from airbyte_cdk.sources.declarative.types import Config, 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 @@ -68,6 +114,12 @@ 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 +from airbyte_cdk.sources.utils.slice_logger import ( + AlwaysLogSliceLogger, + DebugSliceLogger, + SliceLogger, +) +from airbyte_cdk.utils.traced_exception import AirbyteTracedException @dataclass @@ -85,8 +137,33 @@ class TestLimits: max_streams: int = field(default=DEFAULT_MAX_STREAMS) -class ConcurrentDeclarativeSource(ManifestDeclarativeSource, Generic[TState]): - # By default, we defer to a value of 2. A value lower than than could cause a PartitionEnqueuer to be stuck in a state of deadlock +def _get_declarative_component_schema() -> Dict[str, Any]: + try: + raw_component_schema = pkgutil.get_data( + "airbyte_cdk", "sources/declarative/declarative_component_schema.yaml" + ) + if raw_component_schema is not None: + declarative_component_schema = yaml.load(raw_component_schema, Loader=yaml.SafeLoader) + return declarative_component_schema # type: ignore + else: + raise RuntimeError( + "Failed to read manifest component json schema required for deduplication" + ) + except FileNotFoundError as e: + raise FileNotFoundError( + f"Failed to read manifest component json schema required for deduplication: {e}" + ) + + +# todo: AbstractSource can be removed once we've completely moved off all legacy synchronous CDK code paths +# and replaced with implementing the source.py:Source class +# +# todo: The `ConcurrentDeclarativeSource.message_repository()` method can also be removed once AbstractSource +# is no longer inherited from since the only external dependency is from that class. +# +# todo: It is worth investigating removal of the Generic[TState] since it will always be Optional[List[AirbyteStateMessage]] +class ConcurrentDeclarativeSource(AbstractSource, Generic[TState]): + # By default, we defer to a value of 2. A value lower than could cause a PartitionEnqueuer to be stuck in a state of deadlock # because it has hit the limit of futures but not partition reader is consuming them. _LOWEST_SAFE_CONCURRENCY_LEVEL = 2 @@ -104,6 +181,10 @@ def __init__( config_path: Optional[str] = None, **kwargs: Any, ) -> None: + self.logger = logging.getLogger(f"airbyte.{self.name}") + + self._limits = limits + # todo: We could remove state from initialization. Now that streams are grouped during the read(), a source # 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 @@ -132,18 +213,40 @@ def __init__( disable_cache=True if limits else False, ) - self._limits = limits + self._should_normalize = normalize_manifest + self._should_migrate = migrate_manifest + self._declarative_component_schema = _get_declarative_component_schema() + # If custom components are needed, locate and/or register them. + self.components_module: ModuleType | None = get_registered_components_module(config=config) + # set additional attributes + self._debug = debug + self._emit_connector_builder_messages = emit_connector_builder_messages + self._constructor = ( + component_factory + if component_factory + else ModelToComponentFactory( + emit_connector_builder_messages=emit_connector_builder_messages, + max_concurrent_async_job_count=source_config.get("max_concurrent_async_job_count"), + ) + ) - super().__init__( - source_config=source_config, - config=config, - debug=debug, - emit_connector_builder_messages=emit_connector_builder_messages, - migrate_manifest=migrate_manifest, - normalize_manifest=normalize_manifest, - component_factory=component_factory, - config_path=config_path, + self._message_repository = self._constructor.get_message_repository() + self._slice_logger: SliceLogger = ( + AlwaysLogSliceLogger() if emit_connector_builder_messages else DebugSliceLogger() + ) + + # resolve all components in the manifest + self._source_config = self._pre_process_manifest(dict(source_config)) + # validate resolved manifest against the declarative component schema + self._validate_source() + # apply additional post-processing to the manifest + self._post_process_manifest() + + spec: Optional[Mapping[str, Any]] = self._source_config.get("spec") + self._spec_component: Optional[Spec] = ( + self._constructor.create_component(SpecModel, spec, dict()) if spec else None ) + self._config = self._migrate_and_transform_config(config_path, config) or {} concurrency_level_from_manifest = self._source_config.get("concurrency_level") if concurrency_level_from_manifest: @@ -171,15 +274,141 @@ def __init__( logger=self.logger, slice_logger=self._slice_logger, queue=queue, - message_repository=self.message_repository, + message_repository=self._message_repository, + ) + + def _pre_process_manifest(self, manifest: Dict[str, Any]) -> Dict[str, Any]: + """ + Preprocesses the provided manifest dictionary by resolving any manifest references. + + This method modifies the input manifest in place, resolving references using the + ManifestReferenceResolver to ensure all references within the manifest are properly handled. + + Args: + manifest (Dict[str, Any]): The manifest dictionary to preprocess and resolve references in. + + Returns: + None + """ + # For ease of use we don't require the type to be specified at the top level manifest, but it should be included during processing + manifest = self._fix_source_type(manifest) + # Resolve references in the manifest + resolved_manifest = ManifestReferenceResolver().preprocess_manifest(manifest) + # Propagate types and parameters throughout the manifest + propagated_manifest = ManifestComponentTransformer().propagate_types_and_parameters( + "", resolved_manifest, {} ) + return propagated_manifest + + def _fix_source_type(self, manifest: Dict[str, Any]) -> Dict[str, Any]: + """ + Fix the source type in the manifest. This is necessary because the source type is not always set in the manifest. + """ + if "type" not in manifest: + manifest["type"] = "DeclarativeSource" + + return manifest + + def _post_process_manifest(self) -> None: + """ + Post-processes the manifest after validation. + This method is responsible for any additional modifications or transformations needed + after the manifest has been validated and before it is used in the source. + """ + # apply manifest migration, if required + self._migrate_manifest() + # apply manifest normalization, if required + self._normalize_manifest() + + def _migrate_manifest(self) -> None: + """ + This method is used to migrate the manifest. It should be called after the manifest has been validated. + The migration is done in place, so the original manifest is modified. + + The original manifest is returned if any error occurs during migration. + """ + if self._should_migrate: + manifest_migrator = ManifestMigrationHandler(self._source_config) + self._source_config = manifest_migrator.apply_migrations() + # validate migrated manifest against the declarative component schema + self._validate_source() + + def _normalize_manifest(self) -> None: + """ + This method is used to normalize the manifest. It should be called after the manifest has been validated. + + Connector Builder UI rendering requires the manifest to be in a specific format. + - references have been resolved + - the commonly used definitions are extracted to the `definitions.linked.*` + """ + if self._should_normalize: + normalizer = ManifestNormalizer(self._source_config, self._declarative_component_schema) + self._source_config = normalizer.normalize() + + def _validate_source(self) -> None: + """ + Validates the connector manifest against the declarative component schema + """ + + try: + validate(self._source_config, self._declarative_component_schema) + except ValidationError as e: + raise ValidationError( + "Validation against json schema defined in declarative_component_schema.yaml schema failed" + ) from e + + def _migrate_and_transform_config( + self, + config_path: Optional[str], + config: Optional[Config], + ) -> Optional[Config]: + if not config: + return None + if not self._spec_component: + return config + mutable_config = dict(config) + self._spec_component.migrate_config(mutable_config) + if mutable_config != config: + if config_path: + with open(config_path, "w") as f: + json.dump(mutable_config, f) + control_message = create_connector_config_control_message(mutable_config) + print(orjson.dumps(AirbyteMessageSerializer.dump(control_message)).decode()) + self._spec_component.transform_config(mutable_config) + return mutable_config + + def configure(self, config: Mapping[str, Any], temp_dir: str) -> Mapping[str, Any]: + config = self._config or config + return super().configure(config, temp_dir) + + @property + def resolved_manifest(self) -> Mapping[str, Any]: + """ + Returns the resolved manifest configuration for the source. + + This property provides access to the internal source configuration as a mapping, + which contains all settings and parameters required to define the source's behavior. + + Returns: + Mapping[str, Any]: The resolved source configuration manifest. + """ + return self._source_config + + # TODO: Deprecate this class once ConcurrentDeclarativeSource no longer inherits AbstractSource + @property + def message_repository(self) -> MessageRepository: + return self._message_repository + # TODO: Remove this. This property is necessary to safely migrate Stripe during the transition state. @property def is_partially_declarative(self) -> bool: """This flag used to avoid unexpected AbstractStreamFacade processing as concurrent streams.""" return False + def deprecation_warnings(self) -> List[ConnectorBuilderLogMessage]: + return self._constructor.get_model_deprecations() + def read( self, logger: logging.Logger, @@ -237,7 +466,140 @@ def streams(self, config: Mapping[str, Any]) -> List[Union[Stream, AbstractStrea In both case, we will assume that calling the DeclarativeStream is perfectly fine as the result for these is the same regardless of if it is a DeclarativeStream or a DefaultStream (concurrent). This should simply be removed once we have moved away from the mentioned code paths above. """ - return super().streams(config) + + if self._spec_component: + self._spec_component.validate_config(config) + + stream_configs = ( + self._stream_configs(self._source_config, config=config) + self.dynamic_streams + ) + + api_budget_model = self._source_config.get("api_budget") + if api_budget_model: + self._constructor.set_api_budget(api_budget_model, config) + + source_streams = [ + self._constructor.create_component( + ( + StateDelegatingStreamModel + if stream_config.get("type") == StateDelegatingStreamModel.__name__ + else DeclarativeStreamModel + ), + stream_config, + config, + emit_connector_builder_messages=self._emit_connector_builder_messages, + ) + for stream_config in self._initialize_cache_for_parent_streams(deepcopy(stream_configs)) + ] + return source_streams + + @staticmethod + def _initialize_cache_for_parent_streams( + stream_configs: List[Dict[str, Any]], + ) -> List[Dict[str, Any]]: + parent_streams = set() + + def update_with_cache_parent_configs( + parent_configs: list[dict[str, Any]], + ) -> None: + for parent_config in parent_configs: + parent_streams.add(parent_config["stream"]["name"]) + if parent_config["stream"]["type"] == "StateDelegatingStream": + parent_config["stream"]["full_refresh_stream"]["retriever"]["requester"][ + "use_cache" + ] = True + parent_config["stream"]["incremental_stream"]["retriever"]["requester"][ + "use_cache" + ] = True + else: + parent_config["stream"]["retriever"]["requester"]["use_cache"] = True + + for stream_config in stream_configs: + if stream_config.get("incremental_sync", {}).get("parent_stream"): + parent_streams.add(stream_config["incremental_sync"]["parent_stream"]["name"]) + stream_config["incremental_sync"]["parent_stream"]["retriever"]["requester"][ + "use_cache" + ] = True + + elif stream_config.get("retriever", {}).get("partition_router", {}): + partition_router = stream_config["retriever"]["partition_router"] + + if isinstance(partition_router, dict) and partition_router.get( + "parent_stream_configs" + ): + update_with_cache_parent_configs(partition_router["parent_stream_configs"]) + elif isinstance(partition_router, list): + for router in partition_router: + if router.get("parent_stream_configs"): + update_with_cache_parent_configs(router["parent_stream_configs"]) + + for stream_config in stream_configs: + if stream_config["name"] in parent_streams: + if stream_config["type"] == "StateDelegatingStream": + stream_config["full_refresh_stream"]["retriever"]["requester"]["use_cache"] = ( + True + ) + stream_config["incremental_stream"]["retriever"]["requester"]["use_cache"] = ( + True + ) + else: + stream_config["retriever"]["requester"]["use_cache"] = True + return stream_configs + + def spec(self, logger: logging.Logger) -> ConnectorSpecification: + """ + Returns the connector specification (spec) as defined in the Airbyte Protocol. The spec is an object describing the possible + configurations (e.g: username and password) which can be configured when running this connector. For low-code connectors, this + will first attempt to load the spec from the manifest's spec block, otherwise it will load it from "spec.yaml" or "spec.json" + in the project root. + """ + return ( + self._spec_component.generate_spec() if self._spec_component else super().spec(logger) + ) + + def check(self, logger: logging.Logger, config: Mapping[str, Any]) -> AirbyteConnectionStatus: + return super().check(logger, config) + + def check_connection( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> Tuple[bool, Any]: + """ + :param logger: The source logger + :param config: The user-provided configuration as specified by the source's spec. + This usually contains information required to check connection e.g. tokens, secrets and keys etc. + :return: A tuple of (boolean, error). If boolean is true, then the connection check is successful + and we can connect to the underlying data source using the provided configuration. + Otherwise, the input config cannot be used to connect to the underlying data source, + and the "error" object should describe what went wrong. + The error object will be cast to string to display the problem to the user. + """ + return self.connection_checker.check_connection(self, logger, config) + + @property + def connection_checker(self) -> ConnectionChecker: + check = self._source_config["check"] + if "type" not in check: + check["type"] = "CheckStream" + check_stream = self._constructor.create_component( + COMPONENTS_CHECKER_TYPE_MAPPING[check["type"]], + check, + dict(), + emit_connector_builder_messages=self._emit_connector_builder_messages, + ) + if isinstance(check_stream, ConnectionChecker): + return check_stream + else: + raise ValueError( + f"Expected to generate a ConnectionChecker component, but received {check_stream.__class__}" + ) + + @property + def dynamic_streams(self) -> List[Dict[str, Any]]: + return self._dynamic_stream_configs( + manifest=self._source_config, + config=self._config, + with_dynamic_stream_name=True, + ) def _group_streams( self, config: Mapping[str, Any] @@ -333,7 +695,7 @@ def _group_streams( stream_name=declarative_stream.name, schema_loader=declarative_stream._schema_loader, # type: ignore # We are accessing the private property but the public one is optional and we will remove this code soonish retriever=retriever, - message_repository=self.message_repository, + message_repository=self._message_repository, max_records_limit=self._limits.max_records if self._limits else None, @@ -370,7 +732,7 @@ def _group_streams( stream_name=declarative_stream.name, schema_loader=declarative_stream._schema_loader, # type: ignore # We are accessing the private property but the public one is optional and we will remove this code soonish retriever=retriever, - message_repository=self.message_repository, + message_repository=self._message_repository, max_records_limit=self._limits.max_records if self._limits else None, @@ -404,7 +766,7 @@ def _group_streams( stream_name=declarative_stream.name, schema_loader=declarative_stream._schema_loader, # type: ignore # We are accessing the private property but the public one is optional and we will remove this code soonish retriever=declarative_stream.retriever, - message_repository=self.message_repository, + message_repository=self._message_repository, max_records_limit=self._limits.max_records if self._limits else None, ), declarative_stream.retriever.stream_slicer, @@ -416,7 +778,7 @@ def _group_streams( final_state_cursor = FinalStateCursor( stream_name=declarative_stream.name, stream_namespace=declarative_stream.namespace, - message_repository=self.message_repository, + message_repository=self._message_repository, ) concurrent_streams.append( @@ -468,7 +830,7 @@ def _group_streams( stream_name=declarative_stream.name, schema_loader=declarative_stream._schema_loader, # type: ignore # We are accessing the private property but the public one is optional and we will remove this code soonish retriever=retriever, - message_repository=self.message_repository, + message_repository=self._message_repository, max_records_limit=self._limits.max_records if self._limits else None, ), perpartition_cursor, @@ -502,6 +864,127 @@ def _group_streams( return concurrent_streams, synchronous_streams + def _stream_configs( + self, manifest: Mapping[str, Any], config: Mapping[str, Any] + ) -> List[Dict[str, Any]]: + # This has a warning flag for static, but after we finish part 4 we'll replace manifest with self._source_config + stream_configs = [] + for current_stream_config in manifest.get("streams", []): + if ( + "type" in current_stream_config + and current_stream_config["type"] == "ConditionalStreams" + ): + interpolated_boolean = InterpolatedBoolean( + condition=current_stream_config.get("condition"), + parameters={}, + ) + + if interpolated_boolean.eval(config=config): + stream_configs.extend(current_stream_config.get("streams", [])) + else: + if "type" not in current_stream_config: + current_stream_config["type"] = "DeclarativeStream" + stream_configs.append(current_stream_config) + return stream_configs + + def _dynamic_stream_configs( + self, + manifest: Mapping[str, Any], + config: Mapping[str, Any], + with_dynamic_stream_name: Optional[bool] = None, + ) -> List[Dict[str, Any]]: + dynamic_stream_definitions: List[Dict[str, Any]] = manifest.get("dynamic_streams", []) + dynamic_stream_configs: List[Dict[str, Any]] = [] + seen_dynamic_streams: Set[str] = set() + + for dynamic_definition_index, dynamic_definition in enumerate(dynamic_stream_definitions): + components_resolver_config = dynamic_definition["components_resolver"] + + if not components_resolver_config: + raise ValueError( + f"Missing 'components_resolver' in dynamic definition: {dynamic_definition}" + ) + + resolver_type = components_resolver_config.get("type") + if not resolver_type: + raise ValueError( + f"Missing 'type' in components resolver configuration: {components_resolver_config}" + ) + + if resolver_type not in COMPONENTS_RESOLVER_TYPE_MAPPING: + raise ValueError( + f"Invalid components resolver type '{resolver_type}'. " + f"Expected one of {list(COMPONENTS_RESOLVER_TYPE_MAPPING.keys())}." + ) + + if "retriever" in components_resolver_config: + components_resolver_config["retriever"]["requester"]["use_cache"] = True + + # Create a resolver for dynamic components based on type + if resolver_type == "HttpComponentsResolver": + components_resolver = self._constructor.create_component( + model_type=COMPONENTS_RESOLVER_TYPE_MAPPING[resolver_type], + component_definition=components_resolver_config, + config=config, + stream_name=dynamic_definition.get("name"), + ) + else: + components_resolver = self._constructor.create_component( + model_type=COMPONENTS_RESOLVER_TYPE_MAPPING[resolver_type], + component_definition=components_resolver_config, + config=config, + ) + + stream_template_config = dynamic_definition["stream_template"] + + for dynamic_stream in components_resolver.resolve_components( + stream_template_config=stream_template_config + ): + # Get the use_parent_parameters configuration from the dynamic definition + # Default to True for backward compatibility, since connectors were already using it by default when this param was added + use_parent_parameters = dynamic_definition.get("use_parent_parameters", True) + + dynamic_stream = { + **ManifestComponentTransformer().propagate_types_and_parameters( + "", dynamic_stream, {}, use_parent_parameters=use_parent_parameters + ) + } + + if "type" not in dynamic_stream: + dynamic_stream["type"] = "DeclarativeStream" + + # Ensure that each stream is created with a unique name + name = dynamic_stream.get("name") + + if with_dynamic_stream_name: + dynamic_stream["dynamic_stream_name"] = dynamic_definition.get( + "name", f"dynamic_stream_{dynamic_definition_index}" + ) + + if not isinstance(name, str): + raise ValueError( + f"Expected stream name {name} to be a string, got {type(name)}." + ) + + if name in seen_dynamic_streams: + error_message = f"Dynamic streams list contains a duplicate name: {name}. Please contact Airbyte Support." + failure_type = FailureType.system_error + + if resolver_type == "ConfigComponentsResolver": + error_message = f"Dynamic streams list contains a duplicate name: {name}. Please check your configuration." + failure_type = FailureType.config_error + + raise AirbyteTracedException( + message=error_message, + internal_message=error_message, + failure_type=failure_type, + ) + + seen_dynamic_streams.add(name) + dynamic_stream_configs.append(dynamic_stream) + + return dynamic_stream_configs + def _is_concurrent_cursor_incremental_without_partition_routing( self, declarative_stream: DeclarativeStream, diff --git a/airbyte_cdk/sources/declarative/incremental/__init__.py b/airbyte_cdk/sources/declarative/incremental/__init__.py index 395daca6d..5b9bb5143 100644 --- a/airbyte_cdk/sources/declarative/incremental/__init__.py +++ b/airbyte_cdk/sources/declarative/incremental/__init__.py @@ -2,6 +2,10 @@ # Copyright (c) 2022 Airbyte, Inc., all rights reserved. # +from airbyte_cdk.legacy.sources.declarative.incremental.per_partition_cursor import ( + CursorFactory, + PerPartitionCursor, +) from airbyte_cdk.sources.declarative.incremental.concurrent_partition_cursor import ( ConcurrentCursorFactory, ConcurrentPerPartitionCursor, @@ -11,10 +15,6 @@ from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( GlobalSubstreamCursor, ) -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import ( - CursorFactory, - PerPartitionCursor, -) from airbyte_cdk.sources.declarative.incremental.per_partition_with_global import ( PerPartitionWithGlobalCursor, ) diff --git a/airbyte_cdk/sources/declarative/incremental/per_partition_with_global.py b/airbyte_cdk/sources/declarative/incremental/per_partition_with_global.py index c3c816a17..9779b35cc 100644 --- a/airbyte_cdk/sources/declarative/incremental/per_partition_with_global.py +++ b/airbyte_cdk/sources/declarative/incremental/per_partition_with_global.py @@ -3,16 +3,16 @@ # from typing import Any, Iterable, Mapping, MutableMapping, Optional, Union +from airbyte_cdk.legacy.sources.declarative.incremental.per_partition_cursor import ( + CursorFactory, + PerPartitionCursor, +) from airbyte_cdk.sources.declarative.incremental.datetime_based_cursor import DatetimeBasedCursor from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( GlobalSubstreamCursor, iterate_with_last_flag_and_state, ) -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import ( - CursorFactory, - PerPartitionCursor, -) from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter from airbyte_cdk.sources.types import Record, StreamSlice, StreamState diff --git a/airbyte_cdk/sources/declarative/retrievers/retriever.py b/airbyte_cdk/sources/declarative/retrievers/retriever.py index 155de5782..3221f81e0 100644 --- a/airbyte_cdk/sources/declarative/retrievers/retriever.py +++ b/airbyte_cdk/sources/declarative/retrievers/retriever.py @@ -5,9 +5,8 @@ from abc import abstractmethod from typing import Any, Iterable, Mapping, Optional -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import StreamSlice from airbyte_cdk.sources.streams.core import StreamData -from airbyte_cdk.sources.types import StreamState +from airbyte_cdk.sources.types import StreamSlice, StreamState class Retriever: diff --git a/unit_tests/__init__.py b/unit_tests/__init__.py index 51e56f3ad..58b636bf9 100644 --- a/unit_tests/__init__.py +++ b/unit_tests/__init__.py @@ -1,7 +1 @@ -# THIS STOPS SOME MODELS TESTS FROM FALLING OVER. IT'S A HACK, WE SHOULD PIN DOWN WHAT'S ACTUALLY GOING ON HERE - -# Import the thing that needs to be imported to stop the tests from falling over -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource - -# "Use" the thing so that the linter doesn't complain -placeholder = ManifestDeclarativeSource +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/connector_builder/test_property_chunking.py b/unit_tests/connector_builder/test_property_chunking.py index 9998c71fd..ccae4a336 100644 --- a/unit_tests/connector_builder/test_property_chunking.py +++ b/unit_tests/connector_builder/test_property_chunking.py @@ -4,6 +4,7 @@ import copy import json +from datetime import timedelta import freezegun @@ -21,9 +22,12 @@ Level, StreamDescriptor, ) -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) from airbyte_cdk.test.mock_http import HttpMocker, HttpRequest, HttpResponse from airbyte_cdk.test.mock_http.response_builder import find_template +from airbyte_cdk.utils.datetime_helpers import ab_datetime_parse BASE_URL = "https://api.apilayer.com/exchangerates_data/" FREEZE_DATE = "2025-05-23" @@ -196,20 +200,33 @@ @freezegun.freeze_time(f"{FREEZE_DATE}T00:00:00Z") def test_read(): + # We need a state time earlier than the current time otherwise we will not generate any date range + # partitions to process on the concurrent engine + day_before = (ab_datetime_parse(f"{FREEZE_DATE}T00:00:00Z") - timedelta(days=1)).strftime( + "%Y-%m-%d" + ) conversion_base = "USD" config = copy.deepcopy(TEST_READ_CONFIG) - config["start_date"] = f"{FREEZE_DATE}T00:00:00Z" + config["start_date"] = f"{day_before}T00:00:00Z" config["base"] = conversion_base config["api_key"] = "test_api_key" - stream_url = f"{BASE_URL}{FREEZE_DATE}?base={conversion_base}&{PROPERTY_KEY}=" + catalog = ConfiguredAirbyteCatalogSerializer.load(CONFIGURED_CATALOG) + + stream_url = f"{BASE_URL}{day_before}?base={conversion_base}&{PROPERTY_KEY}=" with HttpMocker() as http_mocker: - source = ManifestDeclarativeSource( - source_config=MANIFEST, emit_connector_builder_messages=True - ) limits = TestLimits() + source = ConcurrentDeclarativeSource( + source_config=MANIFEST, + config=config, + catalog=catalog, + state=_A_STATE, + emit_connector_builder_messages=True, + limits=limits, + ) + http_mocker.get( HttpRequest(url=f"{stream_url}{PROPERTY_LIST[0]}%2C{PROPERTY_LIST[1]}"), HttpResponse( @@ -230,7 +247,7 @@ def test_read(): source, "test_read", config, - ConfiguredAirbyteCatalogSerializer.load(CONFIGURED_CATALOG), + catalog, _A_STATE, limits, ) diff --git a/unit_tests/legacy/__init__.py b/unit_tests/legacy/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/unit_tests/legacy/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/legacy/sources/__init__.py b/unit_tests/legacy/sources/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/unit_tests/legacy/sources/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/legacy/sources/declarative/__init__.py b/unit_tests/legacy/sources/declarative/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/unit_tests/legacy/sources/declarative/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/legacy/sources/declarative/incremental/__init__.py b/unit_tests/legacy/sources/declarative/incremental/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/unit_tests/legacy/sources/declarative/incremental/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/sources/declarative/incremental/test_per_partition_cursor.py b/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py similarity index 99% rename from unit_tests/sources/declarative/incremental/test_per_partition_cursor.py rename to unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py index 7e2711007..9cbc1ad1c 100644 --- a/unit_tests/sources/declarative/incremental/test_per_partition_cursor.py +++ b/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor.py @@ -7,15 +7,15 @@ import pytest -from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor -from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( - GlobalSubstreamCursor, -) -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import ( +from airbyte_cdk.legacy.sources.declarative.incremental.per_partition_cursor import ( PerPartitionCursor, PerPartitionKeySerializer, StreamSlice, ) +from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor +from airbyte_cdk.sources.declarative.incremental.global_substream_cursor import ( + GlobalSubstreamCursor, +) from airbyte_cdk.sources.declarative.partition_routers.partition_router import PartitionRouter from airbyte_cdk.sources.types import Record diff --git a/unit_tests/sources/declarative/incremental/test_per_partition_cursor_integration.py b/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor_integration.py similarity index 99% rename from unit_tests/sources/declarative/incremental/test_per_partition_cursor_integration.py rename to unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor_integration.py index 46b726758..f3f8dfc72 100644 --- a/unit_tests/sources/declarative/incremental/test_per_partition_cursor_integration.py +++ b/unit_tests/legacy/sources/declarative/incremental/test_per_partition_cursor_integration.py @@ -7,6 +7,12 @@ import orjson +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 ( AirbyteStateBlob, AirbyteStateMessage, @@ -19,13 +25,8 @@ StreamDescriptor, SyncMode, ) -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import ( - PerPartitionCursor, - StreamSlice, -) -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource from airbyte_cdk.sources.declarative.retrievers.simple_retriever import SimpleRetriever -from airbyte_cdk.sources.types import Record +from airbyte_cdk.sources.types import Record, StreamSlice CURSOR_FIELD = "cursor_field" SYNC_MODE = SyncMode.incremental diff --git a/unit_tests/legacy/sources/declarative/partition_routers/__init__.py b/unit_tests/legacy/sources/declarative/partition_routers/__init__.py new file mode 100644 index 000000000..58b636bf9 --- /dev/null +++ b/unit_tests/legacy/sources/declarative/partition_routers/__init__.py @@ -0,0 +1 @@ +# Copyright (c) 2025 Airbyte, Inc., all rights reserved. diff --git a/unit_tests/sources/declarative/partition_routers/test_parent_state_stream.py b/unit_tests/legacy/sources/declarative/partition_routers/test_parent_state_stream.py similarity index 99% rename from unit_tests/sources/declarative/partition_routers/test_parent_state_stream.py rename to unit_tests/legacy/sources/declarative/partition_routers/test_parent_state_stream.py index 4fbbd7355..c3cab5500 100644 --- a/unit_tests/sources/declarative/partition_routers/test_parent_state_stream.py +++ b/unit_tests/legacy/sources/declarative/partition_routers/test_parent_state_stream.py @@ -8,6 +8,9 @@ import pytest import requests_mock +from airbyte_cdk.legacy.sources.declarative.manifest_declarative_source import ( + ManifestDeclarativeSource, +) from airbyte_cdk.models import ( AirbyteMessage, AirbyteStateBlob, @@ -21,7 +24,6 @@ StreamDescriptor, SyncMode, ) -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource SUBSTREAM_MANIFEST: MutableMapping[str, Any] = { "version": "0.51.42", diff --git a/unit_tests/sources/declarative/test_manifest_declarative_source.py b/unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py similarity index 99% rename from unit_tests/sources/declarative/test_manifest_declarative_source.py rename to unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py index 24258f193..ad6735201 100644 --- a/unit_tests/sources/declarative/test_manifest_declarative_source.py +++ b/unit_tests/legacy/sources/declarative/test_manifest_declarative_source.py @@ -17,6 +17,9 @@ from jsonschema.exceptions import ValidationError import unit_tests.sources.declarative.external_component # Needed for dynamic imports to work +from airbyte_cdk.legacy.sources.declarative.manifest_declarative_source import ( + ManifestDeclarativeSource, +) from airbyte_cdk.models import ( AirbyteLogMessage, AirbyteMessage, @@ -32,7 +35,6 @@ ConcurrentDeclarativeSource, ) from airbyte_cdk.sources.declarative.declarative_stream import DeclarativeStream -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource from airbyte_cdk.sources.declarative.parsers.model_to_component_factory import ( ModelToComponentFactory, ) @@ -67,7 +69,7 @@ def use_external_yaml_spec(self): module = sys.modules[__name__] module_path = os.path.abspath(module.__file__) test_path = os.path.dirname(module_path) - spec_root = test_path.split("/sources/declarative")[0] + spec_root = test_path.split("/legacy/sources/declarative")[0] spec = { "documentationUrl": "https://airbyte.com/#yaml-from-external", @@ -1023,7 +1025,7 @@ def test_source_with_no_external_spec_and_no_in_yaml_spec_fails(self): with pytest.raises(FileNotFoundError): source.spec(logger) - @patch("airbyte_cdk.sources.declarative.declarative_source.DeclarativeSource.read") + @patch("airbyte_cdk.legacy.sources.declarative.declarative_source.DeclarativeSource.read") def test_given_debug_when_read_then_set_log_level(self, declarative_source_read): any_valid_manifest = { "version": "0.29.3", @@ -2258,7 +2260,7 @@ def validate_refs(yaml_file: str) -> List[str]: return invalid_refs yaml_file_path = ( - Path(__file__).resolve().parent.parent.parent.parent + Path(__file__).resolve().parent.parent.parent.parent.parent / "airbyte_cdk/sources/declarative/declarative_component_schema.yaml" ) assert not validate_refs(yaml_file_path) @@ -2428,7 +2430,7 @@ def migration_mocks(monkeypatch): mock_serializer_dump, ) monkeypatch.setattr( - "airbyte_cdk.sources.declarative.manifest_declarative_source.orjson.dumps", + "airbyte_cdk.legacy.sources.declarative.manifest_declarative_source.orjson.dumps", mock_orjson_dumps, ) diff --git a/unit_tests/source_declarative_manifest/test_source_declarative_remote_manifest.py b/unit_tests/source_declarative_manifest/test_source_declarative_remote_manifest.py index a93229a99..76b240870 100644 --- a/unit_tests/source_declarative_manifest/test_source_declarative_remote_manifest.py +++ b/unit_tests/source_declarative_manifest/test_source_declarative_remote_manifest.py @@ -12,7 +12,9 @@ create_declarative_source, handle_command, ) -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) REMOTE_MANIFEST_SPEC_SUBSTRING = '"required":["__injected_declarative_manifest"]' @@ -30,7 +32,7 @@ def test_given_no_injected_declarative_manifest_then_raise_value_error(invalid_r def test_given_injected_declarative_manifest_then_return_declarative_manifest(valid_remote_config): source = create_declarative_source(["check", "--config", str(valid_remote_config)]) - assert isinstance(source, ManifestDeclarativeSource) + assert isinstance(source, ConcurrentDeclarativeSource) def test_parse_manifest_from_file(valid_remote_config: Path) -> None: diff --git a/unit_tests/source_declarative_manifest/test_source_declarative_w_custom_components.py b/unit_tests/source_declarative_manifest/test_source_declarative_w_custom_components.py index 79440254d..e7819624d 100644 --- a/unit_tests/source_declarative_manifest/test_source_declarative_w_custom_components.py +++ b/unit_tests/source_declarative_manifest/test_source_declarative_w_custom_components.py @@ -21,7 +21,9 @@ create_declarative_source, ) from airbyte_cdk.models import ConfiguredAirbyteCatalog, ConfiguredAirbyteStream -from airbyte_cdk.sources.declarative.manifest_declarative_source import ManifestDeclarativeSource +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) from airbyte_cdk.sources.declarative.parsers.custom_code_compiler import ( ENV_VAR_ALLOW_CUSTOM_CODE, INJECTED_COMPONENTS_PY, @@ -251,7 +253,7 @@ def test_sync_with_injected_py_components( source = create_declarative_source( ["check", "--config", temp_config_file.name], ) - assert isinstance(source, ManifestDeclarativeSource) + assert isinstance(source, ConcurrentDeclarativeSource) source.check(logger=logging.getLogger(), config=py_components_config_dict) catalog: AirbyteCatalog = source.discover( logger=logging.getLogger(), config=py_components_config_dict diff --git a/unit_tests/sources/declarative/parsers/test_manifest_normalizer.py b/unit_tests/sources/declarative/parsers/test_manifest_normalizer.py index 99c2da43a..3e934db79 100644 --- a/unit_tests/sources/declarative/parsers/test_manifest_normalizer.py +++ b/unit_tests/sources/declarative/parsers/test_manifest_normalizer.py @@ -3,7 +3,7 @@ # -from airbyte_cdk.sources.declarative.manifest_declarative_source import ( +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( _get_declarative_component_schema, ) from airbyte_cdk.sources.declarative.parsers.manifest_normalizer import ( 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 80c8f1e10..7b09e50dd 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 @@ -8,17 +8,16 @@ import pytest as pytest +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, ) -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import ( - CursorFactory, - PerPartitionCursor, - StreamSlice, -) from airbyte_cdk.sources.declarative.interpolation import InterpolatedString from airbyte_cdk.sources.declarative.partition_routers import ( CartesianProductStreamSlicer, @@ -33,7 +32,7 @@ RequestOptionType, ) from airbyte_cdk.sources.streams.checkpoint import Cursor -from airbyte_cdk.sources.types import Record +from airbyte_cdk.sources.types import Record, StreamSlice from airbyte_cdk.utils import AirbyteTracedException parent_records = [{"id": 1, "data": "data1"}, {"id": 2, "data": "data2"}] diff --git a/unit_tests/sources/declarative/stream_slicers/test_stream_slicer_read_decorator.py b/unit_tests/sources/declarative/stream_slicers/test_stream_slicer_read_decorator.py index 05edf31cf..811f66e5e 100644 --- a/unit_tests/sources/declarative/stream_slicers/test_stream_slicer_read_decorator.py +++ b/unit_tests/sources/declarative/stream_slicers/test_stream_slicer_read_decorator.py @@ -16,9 +16,6 @@ PerPartitionWithGlobalCursor, ) from airbyte_cdk.sources.declarative.incremental.declarative_cursor import DeclarativeCursor -from airbyte_cdk.sources.declarative.incremental.per_partition_cursor import ( - StreamSlice, -) from airbyte_cdk.sources.declarative.interpolation import InterpolatedString from airbyte_cdk.sources.declarative.models import ( CustomRetriever, @@ -38,6 +35,7 @@ StreamSlicerTestReadDecorator, ) from airbyte_cdk.sources.message import NoopMessageRepository +from airbyte_cdk.sources.types import StreamSlice from unit_tests.sources.declarative.async_job.test_integration import MockAsyncJobRepository CURSOR_SLICE_FIELD = "cursor slice field" diff --git a/unit_tests/sources/declarative/test_concurrent_declarative_source.py b/unit_tests/sources/declarative/test_concurrent_declarative_source.py index 4279e53c6..9f96ee50f 100644 --- a/unit_tests/sources/declarative/test_concurrent_declarative_source.py +++ b/unit_tests/sources/declarative/test_concurrent_declarative_source.py @@ -4,17 +4,27 @@ import copy import json +import logging import math +import os +import sys +from copy import deepcopy from datetime import datetime, timedelta, timezone +from pathlib import Path from typing import Any, Dict, Iterable, List, Mapping, Optional, Tuple, Union -from unittest.mock import patch +from unittest.mock import Mock, call, mock_open, patch import freezegun import isodate import pytest +import requests +import yaml +from jsonschema.exceptions import ValidationError from typing_extensions import deprecated +import unit_tests.sources.declarative.external_component # Needed for dynamic imports to work from airbyte_cdk.models import ( + AirbyteLogMessage, AirbyteMessage, AirbyteRecordMessage, AirbyteStateBlob, @@ -26,9 +36,11 @@ ConfiguredAirbyteStream, DestinationSyncMode, FailureType, + Level, Status, StreamDescriptor, SyncMode, + Type, ) from airbyte_cdk.sources.declarative.async_job.job_tracker import ConcurrentJobLimitReached from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( @@ -39,6 +51,7 @@ ClientSideIncrementalRecordFilterDecorator, ) from airbyte_cdk.sources.declarative.partition_routers import AsyncJobPartitionRouter +from airbyte_cdk.sources.declarative.retrievers.simple_retriever import SimpleRetriever from airbyte_cdk.sources.declarative.stream_slicers.declarative_partition_generator import ( StreamSlicerPartitionGenerator, ) @@ -55,6 +68,8 @@ from airbyte_cdk.utils import AirbyteTracedException from airbyte_cdk.utils.datetime_helpers import AirbyteDateTime, ab_datetime_parse +logger = logging.getLogger("airbyte") + _CONFIG = {"start_date": "2024-07-01T00:00:00.000Z"} _CATALOG = ConfiguredAirbyteCatalog( @@ -599,6 +614,14 @@ } +EXTERNAL_CONNECTION_SPECIFICATION = { + "type": "object", + "required": ["api_token"], + "additionalProperties": False, + "properties": {"api_token": {"type": "string"}}, +} + + @deprecated("See note in docstring for more information") class DeclarativeStreamDecorator(Stream): """ @@ -2110,3 +2133,2519 @@ def get_states_for_stream( for message in messages if message.state and message.state.stream.stream_descriptor.name == stream_name ] + + +# The tests below were originally written to test the ManifestDeclarativeSource class. However, +# after deprecating the class when migrating away from legacy synchronous CDK flow, the tests +# were adjusted to validate ConcurrentDeclarativeSource. + + +class MockConcurrentDeclarativeSource(ConcurrentDeclarativeSource): + """ + Mock test class that is needed to monkey patch how we read from various files that make up a declarative source because of how our + tests write configuration files during testing. It is also used to properly namespace where files get written in specific + cases like when we temporarily write files like spec.yaml to the package unit_tests, which is the directory where it will + be read in during the tests. + """ + + +def create_catalog(stream_name: str) -> ConfiguredAirbyteCatalog: + return ConfiguredAirbyteCatalog( + streams=[ + ConfiguredAirbyteStream( + stream=AirbyteStream( + name=stream_name, json_schema={}, supported_sync_modes=[SyncMode.full_refresh] + ), + sync_mode=SyncMode.full_refresh, + destination_sync_mode=DestinationSyncMode.append, + ) + ] + ) + + +class TestConcurrentDeclarativeSource: + @pytest.fixture + def use_external_yaml_spec(self): + # Our way of resolving the absolute path to root of the airbyte-cdk unit test directory where spec.yaml files should + # be written to (i.e. ~/airbyte/airbyte-cdk/python/unit-tests) because that is where they are read from during testing. + module = sys.modules[__name__] + module_path = os.path.abspath(module.__file__) + test_path = os.path.dirname(module_path) + spec_root = test_path.split("/sources/declarative")[0] + + spec = { + "documentationUrl": "https://airbyte.com/#yaml-from-external", + "connectionSpecification": EXTERNAL_CONNECTION_SPECIFICATION, + } + + yaml_path = os.path.join(spec_root, "spec.yaml") + with open(yaml_path, "w") as f: + f.write(yaml.dump(spec)) + yield + os.remove(yaml_path) + + @pytest.fixture + def _base_manifest(self): + """Base manifest without streams or dynamic streams.""" + return { + "version": "3.8.2", + "description": "This is a sample source connector that is very valid.", + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + } + + @pytest.fixture + def _declarative_stream(self): + def declarative_stream_config( + name="lists", requester_type="HttpRequester", custom_requester=None + ): + """Generates a DeclarativeStream configuration.""" + requester_config = { + "type": requester_type, + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + } + if custom_requester: + requester_config.update(custom_requester) + + return { + "type": "DeclarativeStream", + "$parameters": { + "name": name, + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": f"./source_sendgrid/schemas/{{{{ parameters.name }}}}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": requester_config, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + + return declarative_stream_config + + @pytest.fixture + def _dynamic_declarative_stream(self, _declarative_stream): + """Generates a DynamicDeclarativeStream configuration.""" + return { + "type": "DynamicDeclarativeStream", + "stream_template": _declarative_stream(), + "components_resolver": { + "type": "HttpComponentsResolver", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + "components_mapping": [ + { + "type": "ComponentMappingDefinition", + "field_path": ["name"], + "value": "{{ components_value['name'] }}", + } + ], + }, + } + + def test_valid_manifest(self): + manifest = { + "version": "3.8.2", + "definitions": {}, + "description": "This is a sample source connector that is very valid.", + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + { + "type": "DeclarativeStream", + "$parameters": { + "name": "stream_with_custom_requester", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "type": "CustomRequester", + "class_name": "unit_tests.sources.declarative.external_component.SampleCustomComponent", + "path": "/v3/marketing/lists", + "custom_request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + } + assert "unit_tests" in sys.modules + assert "unit_tests.sources" in sys.modules + assert "unit_tests.sources.declarative" in sys.modules + assert "unit_tests.sources.declarative.external_component" in sys.modules + + source = ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + check_stream = source.connection_checker + check_stream.check_connection(source, logging.getLogger(""), {}) + + streams = source.streams({}) + assert len(streams) == 2 + assert isinstance(streams[0], DefaultStream) + assert isinstance(streams[1], DefaultStream) + assert ( + source.resolved_manifest["description"] + == "This is a sample source connector that is very valid." + ) + + def test_manifest_with_spec(self): + manifest = { + "version": "0.29.3", + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + "spec": { + "type": "Spec", + "documentation_url": "https://airbyte.com/#yaml-from-manifest", + "connection_specification": { + "title": "Test Spec", + "type": "object", + "required": ["api_key"], + "additionalProperties": False, + "properties": { + "api_key": { + "type": "string", + "airbyte_secret": True, + "title": "API Key", + "description": "Test API Key", + "order": 0, + } + }, + }, + }, + } + source = ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + connector_specification = source.spec(logger) + assert connector_specification is not None + assert connector_specification.documentationUrl == "https://airbyte.com/#yaml-from-manifest" + assert connector_specification.connectionSpecification["title"] == "Test Spec" + assert connector_specification.connectionSpecification["required"][0] == "api_key" + assert connector_specification.connectionSpecification["additionalProperties"] is False + assert connector_specification.connectionSpecification["properties"]["api_key"] == { + "type": "string", + "airbyte_secret": True, + "title": "API Key", + "description": "Test API Key", + "order": 0, + } + + def test_manifest_with_external_spec(self, use_external_yaml_spec): + manifest = { + "version": "0.29.3", + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + } + source = MockConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + connector_specification = source.spec(logger) + + assert connector_specification.documentationUrl == "https://airbyte.com/#yaml-from-external" + assert connector_specification.connectionSpecification == EXTERNAL_CONNECTION_SPECIFICATION + + def test_source_is_not_created_if_toplevel_fields_are_unknown(self): + manifest = { + "version": "0.29.3", + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + "not_a_valid_field": "error", + } + with pytest.raises(ValidationError): + ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + def test_source_missing_checker_fails_validation(self): + manifest = { + "version": "0.29.3", + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + ], + } + with pytest.raises(ValidationError): + ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + def test_source_with_missing_streams_and_dynamic_streams_fails( + self, _base_manifest, _dynamic_declarative_stream, _declarative_stream + ): + # test case for manifest without streams or dynamic streams + manifest_without_streams_and_dynamic_streams = _base_manifest + with pytest.raises(ValidationError): + ConcurrentDeclarativeSource( + source_config=manifest_without_streams_and_dynamic_streams, + config={}, + catalog=create_catalog("lists"), + state=None, + ) + + # test case for manifest with streams + manifest_with_streams = { + **manifest_without_streams_and_dynamic_streams, + "streams": [ + _declarative_stream(name="lists"), + _declarative_stream( + name="stream_with_custom_requester", + requester_type="CustomRequester", + custom_requester={ + "class_name": "unit_tests.sources.declarative.external_component.SampleCustomComponent", + "custom_request_parameters": {"page_size": 10}, + }, + ), + ], + } + ConcurrentDeclarativeSource( + source_config=manifest_with_streams, + config={}, + catalog=create_catalog("lists"), + state=None, + ) + + # test case for manifest with dynamic streams + manifest_with_dynamic_streams = { + **manifest_without_streams_and_dynamic_streams, + "dynamic_streams": [_dynamic_declarative_stream], + } + ConcurrentDeclarativeSource( + source_config=manifest_with_dynamic_streams, + config={}, + catalog=create_catalog("lists"), + state=None, + ) + + def test_source_with_missing_version_fails(self): + manifest = { + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": 10}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + } + with pytest.raises(ValidationError): + ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + def test_source_with_invalid_stream_config_fails_validation(self): + manifest = { + "version": "0.29.3", + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + } + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + } + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + } + with pytest.raises(ValidationError): + ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + def test_source_with_no_external_spec_and_no_in_yaml_spec_fails(self): + manifest = { + "version": "0.29.3", + "definitions": { + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "lists", + "primary_key": "id", + "url_base": "https://api.sendgrid.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + }, + }, + "requester": { + "path": "/v3/marketing/lists", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + } + ], + "check": {"type": "CheckStream", "stream_names": ["lists"]}, + } + source = ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + # We expect to fail here because we have not created a temporary spec.yaml file + with pytest.raises(FileNotFoundError): + source.spec(logger) + + @pytest.mark.parametrize( + "is_sandbox, expected_stream_count", + [ + pytest.param(True, 3, id="test_sandbox_config_includes_conditional_streams"), + pytest.param(False, 1, id="test_non_sandbox_config_skips_conditional_streams"), + ], + ) + def test_conditional_streams_manifest(self, is_sandbox, expected_stream_count): + manifest = { + "version": "3.8.2", + "definitions": {}, + "description": "This is a sample source connector that is very valid.", + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "students", + "primary_key": "id", + "url_base": "https://api.yasogamihighschool.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_yasogami_high_school/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v1/students", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + { + "type": "ConditionalStreams", + "condition": "{{ config['is_sandbox'] }}", + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "classrooms", + "primary_key": "id", + "url_base": "https://api.yasogamihighschool.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_yasogami_high_school/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v1/classrooms", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + { + "type": "DeclarativeStream", + "$parameters": { + "name": "clubs", + "primary_key": "id", + "url_base": "https://api.yasogamihighschool.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_yasogami_high_school/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v1/clubs", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + ], + }, + ], + "check": {"type": "CheckStream", "stream_names": ["students"]}, + } + + assert "unit_tests" in sys.modules + assert "unit_tests.sources" in sys.modules + assert "unit_tests.sources.declarative" in sys.modules + assert "unit_tests.sources.declarative.external_component" in sys.modules + + config = {"is_sandbox": is_sandbox} + catalog = create_catalog("students") + + source = ConcurrentDeclarativeSource( + source_config=manifest, config=config, catalog=catalog, state=None + ) + + check_stream = source.connection_checker + check_stream.check_connection(source, logging.getLogger(""), config=config) + + actual_streams = source.streams(config=config) + assert len(actual_streams) == expected_stream_count + assert isinstance(actual_streams[0], DefaultStream) + assert actual_streams[0].name == "students" + + if is_sandbox: + assert isinstance(actual_streams[1], DefaultStream) + assert actual_streams[1].name == "classrooms" + assert isinstance(actual_streams[2], DefaultStream) + assert actual_streams[2].name == "clubs" + + assert ( + source.resolved_manifest["description"] + == "This is a sample source connector that is very valid." + ) + + @pytest.mark.parametrize( + "field_to_remove,expected_error", + [ + pytest.param("condition", ValidationError, id="test_no_condition_raises_error"), + pytest.param("streams", ValidationError, id="test_no_streams_raises_error"), + ], + ) + def test_conditional_streams_invalid_manifest(self, field_to_remove, expected_error): + manifest = { + "version": "3.8.2", + "definitions": {}, + "description": "This is a sample source connector that is very valid.", + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "students", + "primary_key": "id", + "url_base": "https://api.yasogamihighschool.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_yasogami_high_school/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v1/students", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + { + "type": "ConditionalStreams", + "condition": "{{ config['is_sandbox'] }}", + "streams": [ + { + "type": "DeclarativeStream", + "$parameters": { + "name": "classrooms", + "primary_key": "id", + "url_base": "https://api.yasogamihighschool.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_yasogami_high_school/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v1/classrooms", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + { + "type": "DeclarativeStream", + "$parameters": { + "name": "clubs", + "primary_key": "id", + "url_base": "https://api.yasogamihighschool.com", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_yasogami_high_school/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 10, + }, + }, + "requester": { + "path": "/v1/clubs", + "authenticator": { + "type": "BearerAuthenticator", + "api_token": "{{ config.apikey }}", + }, + "request_parameters": {"page_size": "{{ 10 }}"}, + }, + "record_selector": {"extractor": {"field_path": ["result"]}}, + }, + }, + ], + }, + ], + "check": {"type": "CheckStream", "stream_names": ["students"]}, + } + + assert "unit_tests" in sys.modules + assert "unit_tests.sources" in sys.modules + assert "unit_tests.sources.declarative" in sys.modules + assert "unit_tests.sources.declarative.external_component" in sys.modules + + del manifest["streams"][1][field_to_remove] + + with pytest.raises(ValidationError): + ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + +def request_log_message(request: dict) -> AirbyteMessage: + return AirbyteMessage( + type=Type.LOG, + log=AirbyteLogMessage(level=Level.INFO, message=f"request:{json.dumps(request)}"), + ) + + +def response_log_message(response: dict) -> AirbyteMessage: + return AirbyteMessage( + type=Type.LOG, + log=AirbyteLogMessage(level=Level.INFO, message=f"response:{json.dumps(response)}"), + ) + + +def _create_request(): + url = "https://example.com/api" + headers = {"Content-Type": "application/json"} + return requests.Request("POST", url, headers=headers, json={"key": "value"}).prepare() + + +def _create_response(body): + response = requests.Response() + response.status_code = 200 + response._content = bytes(json.dumps(body), "utf-8") + response.headers["Content-Type"] = "application/json" + return response + + +def _create_page(response_body): + response = _create_response(response_body) + response.request = _create_request() + return response + + +@pytest.mark.parametrize( + "test_name, manifest, pages, expected_records, expected_calls", + [ + ( + "test_read_manifest_no_pagination_no_partitions", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": {"type": "NoPagination"}, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + } + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page({"rates": [{"ABC": 0}, {"AED": 1}], "_metadata": {"next": "next"}}), + _create_page({"rates": [{"USD": 2}], "_metadata": {"next": "next"}}), + ) + * 10, + [{"ABC": 0}, {"AED": 1}], + [call({}, {}, None)], + ), + ( + "test_read_manifest_with_added_fields", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + }, + "type": "object", + }, + }, + "transformations": [ + { + "type": "AddFields", + "fields": [ + { + "type": "AddedFieldDefinition", + "path": ["added_field_key"], + "value": "added_field_value", + } + ], + } + ], + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": {"type": "NoPagination"}, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + } + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page({"rates": [{"ABC": 0}, {"AED": 1}], "_metadata": {"next": "next"}}), + _create_page({"rates": [{"USD": 2}], "_metadata": {"next": "next"}}), + ) + * 10, + [ + {"ABC": 0, "added_field_key": "added_field_value"}, + {"AED": 1, "added_field_key": "added_field_value"}, + ], + [call({}, {}, None)], + ), + ( + "test_read_manifest_with_flatten_fields", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + }, + "type": "object", + }, + }, + "transformations": [{"type": "FlattenFields"}], + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": {"type": "NoPagination"}, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + } + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page( + { + "rates": [ + {"nested_fields": {"ABC": 0}, "id": 1}, + {"nested_fields": {"AED": 1}, "id": 2}, + ], + "_metadata": {"next": "next"}, + } + ), + _create_page({"rates": [{"USD": 2}], "_metadata": {"next": "next"}}), + ) + * 10, + [ + {"ABC": 0, "id": 1}, + {"AED": 1, "id": 2}, + ], + [call({}, {}, None)], + ), + ( + "test_read_with_pagination_no_partitions", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + "USD": {"type": "number"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": { + "type": "DefaultPaginator", + "page_size": 2, + "page_size_option": { + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"inject_into": "path", "type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 2, + }, + }, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + } + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page({"rates": [{"ABC": 0}, {"AED": 1}], "_metadata": {"next": "next"}}), + _create_page({"rates": [{"USD": 2}], "_metadata": {}}), + ) + * 10, + [{"ABC": 0}, {"AED": 1}, {"USD": 2}], + [ + call({}, {}, None), + call({}, {}, {"next_page_token": "next"}), + ], + ), + ( + "test_no_pagination_with_partition_router", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + "partition": {"type": "number"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "partition_router": { + "type": "ListPartitionRouter", + "values": ["0", "1"], + "cursor_field": "partition", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": {"type": "NoPagination"}, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + } + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page( + { + "rates": [{"ABC": 0, "partition": 0}, {"AED": 1, "partition": 0}], + "_metadata": {"next": "next"}, + } + ), + _create_page( + {"rates": [{"ABC": 2, "partition": 1}], "_metadata": {"next": "next"}} + ), + ), + [{"ABC": 0, "partition": 0}, {"AED": 1, "partition": 0}, {"ABC": 2, "partition": 1}], + [ + call({}, {"partition": "0"}, None), + call({}, {"partition": "1"}, None), + ], + ), + ( + "test_with_pagination_and_partition_router", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + "partition": {"type": "number"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "partition_router": { + "type": "ListPartitionRouter", + "values": ["0", "1"], + "cursor_field": "partition", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": { + "type": "DefaultPaginator", + "page_size": 2, + "page_size_option": { + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"inject_into": "path", "type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 2, + }, + }, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + } + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page( + { + "rates": [{"ABC": 0, "partition": 0}, {"AED": 1, "partition": 0}], + "_metadata": {"next": "next"}, + } + ), + _create_page({"rates": [{"USD": 3, "partition": 0}], "_metadata": {}}), + _create_page({"rates": [{"ABC": 2, "partition": 1}], "_metadata": {}}), + ), + [ + {"ABC": 0, "partition": 0}, + {"AED": 1, "partition": 0}, + {"USD": 3, "partition": 0}, + {"ABC": 2, "partition": 1}, + ], + [ + call({}, {"partition": "0"}, None), + call({}, {"partition": "0"}, {"next_page_token": "next"}), + call({}, {"partition": "1"}, None), + ], + ), + ], +) +def test_read_concurrent_declarative_source( + test_name, manifest, pages, expected_records, expected_calls +): + _stream_name = "Rates" + with patch.object(SimpleRetriever, "_fetch_next_page", side_effect=pages) as mock_retriever: + output_data = [ + message.record.data for message in _run_read(manifest, _stream_name) if message.record + ] + assert output_data == expected_records + mock_retriever.assert_has_calls(expected_calls) + + +def test_only_parent_streams_use_cache(): + applications_stream = { + "type": "DeclarativeStream", + "$parameters": { + "name": "applications", + "primary_key": "id", + "url_base": "https://harvest.greenhouse.io/v1/", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "per_page", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ headers['link']['next']['url'] }}", + "stop_condition": "{{ 'next' not in headers['link'] }}", + "page_size": 100, + }, + }, + "requester": { + "path": "applications", + "authenticator": { + "type": "BasicHttpAuthenticator", + "username": "{{ config['api_key'] }}", + }, + }, + "record_selector": {"extractor": {"type": "DpathExtractor", "field_path": []}}, + }, + } + + manifest = { + "version": "0.29.3", + "definitions": {}, + "streams": [ + deepcopy(applications_stream), + { + "type": "DeclarativeStream", + "$parameters": { + "name": "applications_interviews", + "primary_key": "id", + "url_base": "https://harvest.greenhouse.io/v1/", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "per_page", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ headers['link']['next']['url'] }}", + "stop_condition": "{{ 'next' not in headers['link'] }}", + "page_size": 100, + }, + }, + "requester": { + "path": "applications_interviews", + "authenticator": { + "type": "BasicHttpAuthenticator", + "username": "{{ config['api_key'] }}", + }, + }, + "record_selector": {"extractor": {"type": "DpathExtractor", "field_path": []}}, + "partition_router": { + "parent_stream_configs": [ + { + "parent_key": "id", + "partition_field": "parent_id", + "stream": deepcopy(applications_stream), + } + ], + "type": "SubstreamPartitionRouter", + }, + }, + }, + { + "type": "DeclarativeStream", + "$parameters": { + "name": "jobs", + "primary_key": "id", + "url_base": "https://harvest.greenhouse.io/v1/", + }, + "schema_loader": { + "name": "{{ parameters.stream_name }}", + "file_path": "./source_sendgrid/schemas/{{ parameters.name }}.yaml", + }, + "retriever": { + "paginator": { + "type": "DefaultPaginator", + "page_size": 10, + "page_size_option": { + "type": "RequestOption", + "inject_into": "request_parameter", + "field_name": "per_page", + }, + "page_token_option": {"type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ headers['link']['next']['url'] }}", + "stop_condition": "{{ 'next' not in headers['link'] }}", + "page_size": 100, + }, + }, + "requester": { + "path": "jobs", + "authenticator": { + "type": "BasicHttpAuthenticator", + "username": "{{ config['api_key'] }}", + }, + }, + "record_selector": {"extractor": {"type": "DpathExtractor", "field_path": []}}, + }, + }, + ], + "check": {"type": "CheckStream", "stream_names": ["applications"]}, + } + source = ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + + streams = source.streams({}) + assert len(streams) == 3 + + # Main stream with caching (parent for substream `applications_interviews`) + stream_0 = streams[0] + assert stream_0.name == "applications" + assert isinstance(stream_0, DefaultStream) + assert stream_0._stream_partition_generator._partition_factory._retriever.requester.use_cache + + # Substream + stream_1 = streams[1] + assert stream_1.name == "applications_interviews" + assert isinstance(stream_1, DefaultStream) + assert ( + not stream_1._stream_partition_generator._partition_factory._retriever.requester.use_cache + ) + + # Parent stream created for substream + assert ( + stream_1._stream_partition_generator._partition_factory._retriever.stream_slicer.parent_stream_configs[ + 0 + ].stream.name + == "applications" + ) + assert stream_1._stream_partition_generator._partition_factory._retriever.stream_slicer.parent_stream_configs[ + 0 + ].stream.retriever.requester.use_cache + + # Main stream without caching + stream_2 = streams[2] + assert stream_2.name == "jobs" + assert isinstance(stream_2, DefaultStream) + assert ( + not stream_2._stream_partition_generator._partition_factory._retriever.requester.use_cache + ) + + +def _run_read(manifest: Mapping[str, Any], stream_name: str) -> List[AirbyteMessage]: + source = ConcurrentDeclarativeSource( + source_config=manifest, config={}, catalog=create_catalog("lists"), state=None + ) + catalog = ConfiguredAirbyteCatalog( + streams=[ + ConfiguredAirbyteStream( + stream=AirbyteStream( + name=stream_name, json_schema={}, supported_sync_modes=[SyncMode.full_refresh] + ), + sync_mode=SyncMode.full_refresh, + destination_sync_mode=DestinationSyncMode.append, + ) + ] + ) + return list(source.read(logger, {}, catalog, {})) + + +def test_declarative_component_schema_valid_ref_links(): + def load_yaml(file_path) -> Mapping[str, Any]: + with open(file_path, "r") as file: + return yaml.safe_load(file) + + def extract_refs(data, base_path="#") -> List[str]: + refs = [] + if isinstance(data, dict): + for key, value in data.items(): + if key == "$ref" and isinstance(value, str) and value.startswith("#"): + ref_path = value + refs.append(ref_path) + else: + refs.extend(extract_refs(value, base_path)) + elif isinstance(data, list): + for item in data: + refs.extend(extract_refs(item, base_path)) + return refs + + def resolve_pointer(data: Mapping[str, Any], pointer: str) -> bool: + parts = pointer.split("/")[1:] # Skip the first empty part due to leading '#/' + current = data + try: + for part in parts: + part = part.replace("~1", "/").replace("~0", "~") # Unescape JSON Pointer + current = current[part] + return True + except (KeyError, TypeError): + return False + + def validate_refs(yaml_file: str) -> List[str]: + data = load_yaml(yaml_file) + refs = extract_refs(data) + invalid_refs = [ref for ref in refs if not resolve_pointer(data, ref.replace("#", ""))] + return invalid_refs + + yaml_file_path = ( + Path(__file__).resolve().parent.parent.parent.parent + / "airbyte_cdk/sources/declarative/declarative_component_schema.yaml" + ) + assert not validate_refs(yaml_file_path) + + +@pytest.mark.parametrize( + "test_name, manifest, pages, expected_states_qty", + [ + ( + "test_with_pagination_and_partition_router", + { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Rates"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Rates", + "primary_key": [], + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": { + "$schema": "http://json-schema.org/schema#", + "properties": { + "ABC": {"type": "number"}, + "AED": {"type": "number"}, + "partition": {"type": "number"}, + }, + "type": "object", + }, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://api.apilayer.com", + "path": "/exchangerates_data/latest", + "http_method": "GET", + "request_parameters": {}, + "request_headers": {}, + "request_body_json": {}, + "authenticator": { + "type": "ApiKeyAuthenticator", + "header": "apikey", + "api_token": "{{ config['api_key'] }}", + }, + }, + "partition_router": { + "type": "ListPartitionRouter", + "values": ["0", "1"], + "cursor_field": "partition", + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": ["rates"]}, + }, + "paginator": { + "type": "DefaultPaginator", + "page_size": 2, + "page_size_option": { + "inject_into": "request_parameter", + "field_name": "page_size", + }, + "page_token_option": {"inject_into": "path", "type": "RequestPath"}, + "pagination_strategy": { + "type": "CursorPagination", + "cursor_value": "{{ response._metadata.next }}", + "page_size": 2, + }, + }, + }, + "incremental_sync": { + "type": "DatetimeBasedCursor", + "cursor_datetime_formats": ["%Y-%m-%dT%H:%M:%S.%fZ"], + "datetime_format": "%Y-%m-%dT%H:%M:%S.%fZ", + "cursor_field": "updated_at", + "start_datetime": { + "datetime": "{{ config.get('start_date', '2020-10-16T00:00:00.000Z') }}" + }, + }, + } + ], + "spec": { + "connection_specification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["api_key"], + "properties": { + "api_key": { + "type": "string", + "title": "API Key", + "airbyte_secret": True, + }, + "start_date": { + "title": "Start Date", + "description": "UTC date and time in the format YYYY-MM-DDTHH:MM:SS.000Z. During incremental sync, any data generated before this date will not be replicated. If left blank, the start date will be set to 2 years before the present date.", + "pattern": "^[0-9]{4}-[0-9]{2}-[0-9]{2}T[0-9]{2}:[0-9]{2}:[0-9]{2}Z$", + "pattern_descriptor": "YYYY-MM-DDTHH:MM:SS.000Z", + "examples": ["2020-11-16T00:00:00.000Z"], + "type": "string", + "format": "date-time", + }, + }, + "additionalProperties": True, + }, + "documentation_url": "https://example.org", + "type": "Spec", + }, + }, + ( + _create_page( + { + "rates": [ + {"ABC": 0, "partition": 0, "updated_at": "2020-11-16T00:00:00.000Z"}, + {"AED": 1, "partition": 0, "updated_at": "2020-11-16T00:00:00.000Z"}, + ], + "_metadata": {"next": "next"}, + } + ), + _create_page( + { + "rates": [ + {"USD": 3, "partition": 0, "updated_at": "2020-11-16T00:00:00.000Z"} + ], + "_metadata": {}, + } + ), + _create_page( + { + "rates": [ + {"ABC": 2, "partition": 1, "updated_at": "2020-11-16T00:00:00.000Z"} + ], + "_metadata": {}, + } + ), + ), + 2, + ), + ], +) +def test_slice_checkpoint(test_name, manifest, pages, expected_states_qty): + _stream_name = "Rates" + with patch.object(SimpleRetriever, "_fetch_next_page", side_effect=pages): + states = [message.state for message in _run_read(manifest, _stream_name) if message.state] + assert len(states) == expected_states_qty + + +@pytest.fixture +def migration_mocks(monkeypatch): + mock_message_repository = Mock() + mock_message_repository.consume_queue.return_value = [Mock()] + + _mock_open = mock_open() + mock_json_dump = Mock() + mock_print = Mock() + mock_serializer_dump = Mock() + + mock_decoded_bytes = Mock() + mock_decoded_bytes.decode.return_value = "decoded_message" + mock_orjson_dumps = Mock(return_value=mock_decoded_bytes) + + monkeypatch.setattr("builtins.open", _mock_open) + monkeypatch.setattr("json.dump", mock_json_dump) + monkeypatch.setattr("builtins.print", mock_print) + monkeypatch.setattr( + "airbyte_cdk.models.airbyte_protocol_serializers.AirbyteMessageSerializer.dump", + mock_serializer_dump, + ) + monkeypatch.setattr( + "airbyte_cdk.sources.declarative.concurrent_declarative_source.orjson.dumps", + mock_orjson_dumps, + ) + + return { + "message_repository": mock_message_repository, + "open": _mock_open, + "json_dump": mock_json_dump, + "print": mock_print, + "serializer_dump": mock_serializer_dump, + "orjson_dumps": mock_orjson_dumps, + "decoded_bytes": mock_decoded_bytes, + } + + +def test_given_unmigrated_config_when_migrating_then_config_is_migrated(migration_mocks) -> None: + input_config = {"planet": "CRSC"} + + manifest = { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Test"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Test", + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": {"type": "object"}, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://example.org", + "path": "/test", + "authenticator": {"type": "NoAuth"}, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": []}, + }, + }, + } + ], + "spec": { + "type": "Spec", + "documentation_url": "https://example.org", + "connection_specification": {}, + "config_normalization_rules": { + "type": "ConfigNormalizationRules", + "config_migrations": [ + { + "type": "ConfigMigration", + "description": "Test migration", + "transformations": [ + { + "type": "ConfigRemapField", + "map": {"CRSC": "Coruscant"}, + "field_path": ["planet"], + } + ], + } + ], + }, + }, + } + + ConcurrentDeclarativeSource( + source_config=manifest, + config=input_config, + config_path="/fake/config/path", + catalog=create_catalog("lists"), + state=None, + ) + + migration_mocks["open"].assert_called_once_with("/fake/config/path", "w") + migration_mocks["json_dump"].assert_called_once() + migration_mocks["print"].assert_called() + migration_mocks["serializer_dump"].assert_called() + migration_mocks["orjson_dumps"].assert_called() + migration_mocks["decoded_bytes"].decode.assert_called() + + +def test_given_already_migrated_config_no_control_message_is_emitted(migration_mocks) -> None: + input_config = {"planet": "Coruscant"} + + manifest = { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Test"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Test", + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": {"type": "object"}, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://example.org", + "path": "/test", + "authenticator": {"type": "NoAuth"}, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": []}, + }, + }, + } + ], + "spec": { + "type": "Spec", + "documentation_url": "https://example.org", + "connection_specification": {}, + "config_normalization_rules": { + "type": "ConfigNormalizationRules", + "config_migrations": [ + { + "type": "ConfigMigration", + "description": "Test migration", + "transformations": [ + { + "type": "ConfigRemapField", + "map": {"CRSC": "Coruscant"}, + "field_path": ["planet"], + } + ], + } + ], + }, + }, + } + + ConcurrentDeclarativeSource( + source_config=manifest, + config=input_config, + config_path="/fake/config/path", + catalog=create_catalog("lists"), + state=None, + ) + + migration_mocks["open"].assert_not_called() + migration_mocks["json_dump"].assert_not_called() + migration_mocks["print"].assert_not_called() + migration_mocks["serializer_dump"].assert_not_called() + migration_mocks["orjson_dumps"].assert_not_called() + migration_mocks["decoded_bytes"].decode.assert_not_called() + + +def test_given_transformations_config_is_transformed(): + input_config = {"planet": "CRSC"} + + manifest = { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Test"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Test", + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": {"type": "object"}, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://example.org", + "path": "/test", + "authenticator": {"type": "NoAuth"}, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": []}, + }, + }, + } + ], + "spec": { + "type": "Spec", + "documentation_url": "https://example.org", + "connection_specification": {}, + "config_normalization_rules": { + "type": "ConfigNormalizationRules", + "transformations": [ + { + "type": "ConfigAddFields", + "fields": [ + { + "type": "AddedFieldDefinition", + "path": ["population"], + "value": "{{ config['planet'] }}", + } + ], + }, + { + "type": "ConfigRemapField", + "map": {"CRSC": "Coruscant"}, + "field_path": ["planet"], + }, + { + "type": "ConfigRemapField", + "map": {"CRSC": 3_000_000_000_000}, + "field_path": ["population"], + }, + ], + }, + }, + } + + source = ConcurrentDeclarativeSource( + source_config=manifest, + config=input_config, + catalog=create_catalog("lists"), + state=None, + ) + + source.write_config = Mock(return_value=None) + + config = source.configure(input_config, "/fake/temp/dir") + + assert config != input_config + assert config == {"planet": "Coruscant", "population": 3_000_000_000_000} + + +def test_given_valid_config_streams_validates_config_and_does_not_raise(): + input_config = {"schema_to_validate": {"planet": "Coruscant"}} + + manifest = { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Test"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Test", + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": {"type": "object"}, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://example.org", + "path": "/test", + "authenticator": {"type": "NoAuth"}, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": []}, + }, + }, + } + ], + "spec": { + "type": "Spec", + "documentation_url": "https://example.org", + "connection_specification": {}, + "parameters": {}, + "config_normalization_rules": { + "type": "ConfigNormalizationRules", + "validations": [ + { + "type": "DpathValidator", + "field_path": ["schema_to_validate"], + "validation_strategy": { + "type": "ValidateAdheresToSchema", + "base_schema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "Test Spec", + "type": "object", + "properties": {"planet": {"type": "string"}}, + "required": ["planet"], + "additionalProperties": False, + }, + }, + } + ], + }, + }, + } + + source = ConcurrentDeclarativeSource( + source_config=manifest, + config=input_config, + catalog=create_catalog("lists"), + state=None, + ) + + source.streams(input_config) + + +def test_given_invalid_config_streams_validates_config_and_raises(): + input_config = {"schema_to_validate": {"will_fail": "Coruscant"}} + + manifest = { + "version": "0.34.2", + "type": "DeclarativeSource", + "check": {"type": "CheckStream", "stream_names": ["Test"]}, + "streams": [ + { + "type": "DeclarativeStream", + "name": "Test", + "schema_loader": { + "type": "InlineSchemaLoader", + "schema": {"type": "object"}, + }, + "retriever": { + "type": "SimpleRetriever", + "requester": { + "type": "HttpRequester", + "url_base": "https://example.org", + "path": "/test", + "authenticator": {"type": "NoAuth"}, + }, + "record_selector": { + "type": "RecordSelector", + "extractor": {"type": "DpathExtractor", "field_path": []}, + }, + }, + } + ], + "spec": { + "type": "Spec", + "documentation_url": "https://example.org", + "connection_specification": {}, + "parameters": {}, + "config_normalization_rules": { + "type": "ConfigNormalizationRules", + "validations": [ + { + "type": "DpathValidator", + "field_path": ["schema_to_validate"], + "validation_strategy": { + "type": "ValidateAdheresToSchema", + "base_schema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "Test Spec", + "type": "object", + "properties": {"planet": {"type": "string"}}, + "required": ["planet"], + "additionalProperties": False, + }, + }, + } + ], + }, + }, + } + source = ConcurrentDeclarativeSource( + source_config=manifest, + config=input_config, + catalog=create_catalog("lists"), + state=None, + ) + + with pytest.raises(ValueError): + source.streams(input_config) diff --git a/unit_tests/test/test_standard_tests.py b/unit_tests/test/test_standard_tests.py index aa2d38a2b..d5dd28277 100644 --- a/unit_tests/test/test_standard_tests.py +++ b/unit_tests/test/test_standard_tests.py @@ -5,7 +5,9 @@ import pytest -from airbyte_cdk.sources.declarative.declarative_source import DeclarativeSource +from airbyte_cdk.sources.declarative.concurrent_declarative_source import ( + ConcurrentDeclarativeSource, +) from airbyte_cdk.sources.source import Source from airbyte_cdk.test.standard_tests._job_runner import IConnector @@ -13,7 +15,7 @@ @pytest.mark.parametrize( "input, expected", [ - (DeclarativeSource, True), + (ConcurrentDeclarativeSource, True), (Source, True), (None, False), ("", False),