From e6d93ce024dcbfe894dbdd244a20a5095ce5bb9f Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 21 Dec 2023 21:50:51 -0500 Subject: [PATCH 01/74] Working: new async_http.py module --- .../sources/streams/http/async_http.py | 434 ++++++++++++++++++ 1 file changed, 434 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py new file mode 100644 index 000000000000..6ebe8c978706 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py @@ -0,0 +1,434 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import json +import logging +import os +from abc import ABC, abstractmethod +from pathlib import Path +from typing import Any, Callable, Iterable, List, Mapping, Optional, Tuple, TypeVar, Union +from yarl import URL + +import aiohttp +import aiohttp_client_cache +from aiohttp import BasicAuth +from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE +from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession +from airbyte_cdk.sources.streams.call_rate import APIBudget +from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.streams.http import HttpStream +from airbyte_cdk.sources.utils.types import JsonType +from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH + +from .auth.core import HttpAuthenticator, NoAuth +from .exceptions import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException +from .rate_limiting import default_backoff_handler, user_defined_backoff_handler + +# list of all possible HTTP methods which can be used for sending of request bodies +BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") +T = TypeVar("T") + + +class AsyncHttpStream(HttpStream, ABC): + """ + Base abstract class for an Airbyte Stream using the HTTP protocol with asyncio. + + Basic building block for users building an Airbyte source for an async HTTP API. + """ + + # TODO: remove legacy HttpAuthenticator authenticator references + def __init__(self, authenticator: Optional[Union[BasicAuth, HttpAuthenticator]] = None, api_budget: Optional[APIBudget] = None): + self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) + self._session: aiohttp.ClientSession = None + assert authenticator + self._authenticator = authenticator # TODO: handle the preexisting code paths + + def request_session(self) -> aiohttp.ClientSession: + """ + Session factory based on use_cache property and call rate limits (api_budget parameter) + :return: instance of request-based session + """ + if self.use_cache: + cache_dir = os.getenv(ENV_REQUEST_CACHE_PATH) + # Use in-memory cache if cache_dir is not set + # This is a non-obvious interface, but it ensures we don't write sql files when running unit tests + if cache_dir: + sqlite_path = str(Path(cache_dir) / self.cache_filename) + else: + sqlite_path = "file::memory:?cache=shared" + return AsyncCachedLimiterSession(sqlite_path, backend="sqlite", api_budget=self._api_budget) # type: ignore # there are no typeshed stubs for requests_cache + else: + return AsyncLimiterSession(api_budget=self._api_budget) + + def clear_cache(self) -> None: + """ + Clear cached requests for current session, can be called any time + """ + if isinstance(self._session, aiohttp_client_cache.CachedSession): + self._session.cache.clear() + + @abstractmethod + async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + """ + Override this method to define a pagination strategy. + + The value returned from this method is passed to most other methods in this class. Use it to form a request e.g: set headers or query params. + + :return: The token for the next page from the input response object. Returning None means there are no more pages to read in this response. + """ + + @abstractmethod + async def parse_response( + self, + response: aiohttp.ClientResponse, + *, + stream_state: Mapping[str, Any], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> List[Mapping[str, Any]]: + """ + Parses the raw response object into a list of records. + By default, this returns an iterable containing the input. Override to parse differently. + :param response: + :param stream_state: + :param stream_slice: + :param next_page_token: + :return: An iterable containing the parsed response + """ + + # TODO move all the retry logic to a functor/decorator which is input as an init parameter + def should_retry(self, response: aiohttp.ClientResponse) -> bool: + """ + Override to set different conditions for backoff based on the response from the server. + + By default, back off on the following HTTP response statuses: + - 429 (Too Many Requests) indicating rate limiting + - 500s to handle transient server errors + + Unexpected but transient exceptions (connection timeout, DNS resolution failed, etc..) are retried by default. + """ + return response.status == 429 or 500 <= response.status < 600 + + def backoff_time(self, response: aiohttp.ClientResponse) -> Optional[float]: + """ + Override this method to dynamically determine backoff time e.g: by reading the X-Retry-After header. + + This method is called only if should_backoff() returns True for the input request. + + :param response: + :return how long to backoff in seconds. The return value may be a floating point number for subsecond precision. Returning None defers backoff + to the default backoff behavior (e.g using an exponential algorithm). + """ + return None + + def error_message(self, response: aiohttp.ClientResponse) -> str: + """ + Override this method to specify a custom error message which can incorporate the HTTP response received + + :param response: The incoming HTTP response from the partner API + :return: + """ + return "" + + def _create_prepared_request( + self, + path: str, + headers: Optional[Mapping[str, str]] = None, + params: Optional[Mapping[str, str]] = None, + json: Optional[Mapping[str, Any]] = None, + data: Optional[Union[str, Mapping[str, Any]]] = None, + ) -> aiohttp.ClientRequest: + return self._create_aiohttp_client_request(path, headers, params, json, data) + + def _create_aiohttp_client_request( + self, + path: str, + headers: Optional[Mapping[str, str]] = None, + params: Optional[Mapping[str, str]] = None, + json_data: Optional[Mapping[str, Any]] = None, + data: Optional[Union[str, Mapping[str, Any]]] = None, + ) -> aiohttp.ClientRequest: + str_url = "http://localhost:8000" + url = URL(str_url) + if self.must_deduplicate_query_params(): + query_params = self.deduplicate_query_params(str_url, params) + else: + query_params = params or {} + if self.http_method.upper() in BODY_REQUEST_METHODS: + if json_data and data: + raise RequestBodyException( + "At the same time only one of the 'request_body_data' and 'request_body_json' functions can return data" + ) + + client_request = aiohttp.ClientRequest( + self.http_method, url, headers=headers, params=query_params, data=json.dumps(json_data) if json_data else data + ) # TODO: add json header if json_data? + + return client_request + + async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any]) -> aiohttp.ClientResponse: + """ + Wraps sending the request in rate limit and error handlers. + Please note that error handling for HTTP status codes will be ignored if raise_on_http_errors is set to False + + This method handles two types of exceptions: + 1. Expected transient exceptions e.g: 429 status code. + 2. Unexpected transient exceptions e.g: timeout. + + To trigger a backoff, we raise an exception that is handled by the backoff decorator. If an exception is not handled by the decorator will + fail the sync. + + For expected transient exceptions, backoff time is determined by the type of exception raised: + 1. CustomBackoffException uses the user-provided backoff value + 2. DefaultBackoffException falls back on the decorator's default behavior e.g: exponential backoff + + Unexpected transient exceptions use the default backoff parameters. + Unexpected persistent exceptions are not handled and will cause the sync to fail. + """ + self.logger.debug( + "Making outbound API request", extra={"headers": request.headers, "url": request.url, "request_body": request.body} + ) + + # TODO: get headers and anything else off of request & combine with request_kwargs? + response = await self._session.request(request.method, request.url, **request_kwargs) + + # Evaluation of response.text can be heavy, for example, if streaming a large response + # Do it only in debug mode + if self.logger.isEnabledFor(logging.DEBUG): + self.logger.debug( + "Receiving response", extra={"headers": response.headers, "status": response.status, "body": response.text} + ) + if self.should_retry(response): + custom_backoff_time = self.backoff_time(response) + error_message = self.error_message(response) + if custom_backoff_time: + raise UserDefinedBackoffException( + backoff=custom_backoff_time, response=response, error_message=error_message + ) + else: + raise DefaultBackoffException(response=response, error_message=error_message) + elif self.raise_on_http_errors: + # Raise any HTTP exceptions that happened in case there were unexpected ones + try: + response.raise_for_status() + except aiohttp.ClientResponseError as exc: + self.logger.error(response.text) + raise exc + return response + + async def _ensure_session(self) -> aiohttp.ClientSession: + + # if self.use_cache: + # cache_dir = os.getenv(ENV_REQUEST_CACHE_PATH) + # # Use in-memory cache if cache_dir is not set + # # This is a non-obvious interface, but it ensures we don't write sql files when running unit tests + # if cache_dir: + # sqlite_path = str(Path(cache_dir) / self.cache_filename) + # else: + # sqlite_path = "file::memory:?cache=shared" + # self._session = AsyncCachedLimiterSession(sqlite_path, backend="sqlite", api_budget=self._api_budget) # type: ignore # there are no typeshed stubs for requests_cache + # else: + # self._session = AsyncLimiterSession(api_budget=self._api_budget) + + if self._session is None or self._session.closed: # TODO: why is the session closing? + # TODO: figure out caching of session & requests + connector = aiohttp.TCPConnector( + limit_per_host=MAX_CONNECTION_POOL_SIZE, + limit=MAX_CONNECTION_POOL_SIZE, + ) + kwargs = {} + assert self._authenticator.get_auth_header() + if self._authenticator: + kwargs['headers'] = self._authenticator.get_auth_header() + self._session = aiohttp.ClientSession(connector=connector, **kwargs) + return self._session + + async def _send_request(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any]) -> aiohttp.ClientResponse: + """ + Creates backoff wrappers which are responsible for retry logic + """ + + """ + Backoff package has max_tries parameter that means total number of + tries before giving up, so if this number is 0 no calls expected to be done. + But for this class we call it max_REtries assuming there would be at + least one attempt and some retry attempts, to comply this logic we add + 1 to expected retries attempts. + """ + max_tries = self.max_retries + """ + According to backoff max_tries docstring: + max_tries: The maximum number of attempts to make before giving + up ...The default value of None means there is no limit to + the number of tries. + This implies that if max_tries is explicitly set to None there is no + limit to retry attempts, otherwise it is limited number of tries. But + this is not true for current version of backoff packages (1.8.0). Setting + max_tries to 0 or negative number would result in endless retry attempts. + Add this condition to avoid an endless loop if it hasn't been set + explicitly (i.e. max_retries is not None). + """ + max_time = self.max_time + """ + According to backoff max_time docstring: + max_time: The maximum total amount of time to try for before + giving up. Once expired, the exception will be allowed to + escape. If a callable is passed, it will be + evaluated at runtime and its return value used. + """ + if max_tries is not None: + max_tries = max(0, max_tries) + 1 + assert not self._session.closed + + @default_backoff_handler(max_tries=max_tries, max_time=max_time, factor=self.retry_factor) + @user_defined_backoff_handler(max_tries=max_tries, max_time=max_time) + async def send(): + return await self._send(request, request_kwargs) + + return await send() + + @classmethod + async def parse_response_error_message(cls, response: aiohttp.ClientResponse) -> Optional[str]: # TODO + """ + Parses the raw response object from a failed request into a user-friendly error message. + By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. + + :param response: + :return: A user-friendly message that indicates the cause of the error + """ + + # default logic to grab error from common fields + def _try_get_error(value: Optional[JsonType]) -> Optional[str]: + if isinstance(value, str): + return value + elif isinstance(value, list): + errors_in_value = [_try_get_error(v) for v in value] + return ", ".join(v for v in errors_in_value if v is not None) + elif isinstance(value, dict): + new_value = ( + value.get("message") + or value.get("messages") + or value.get("error") + or value.get("errors") + or value.get("failures") + or value.get("failure") + or value.get("detail") + ) + return _try_get_error(new_value) + return None + + try: + body = await response.json() + return _try_get_error(body) + except json.JSONDecodeError: + return None + + async def get_error_display_message(self, exception: BaseException) -> Optional[str]: + """ + Retrieves the user-friendly display message that corresponds to an exception. + This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. + + The default implementation of this method only handles HTTPErrors by passing the response to self.parse_response_error_message(). + The method should be overriden as needed to handle any additional exception types. + + :param exception: The exception that was raised + :return: A user-friendly message that indicates the cause of the error + """ + if isinstance(exception, aiohttp.ClientResponseError) and exception.message is not None: + return await self.parse_response_error_message(exception) + return None + + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> Iterable[StreamData]: + + async for record in self._read_pages( + lambda req, res, state, _slice: self.parse_response(res, stream_slice=_slice, stream_state=state), stream_slice, + stream_state + ): + yield record + + async def _read_pages( + self, + records_generator_fn: Callable[ + [aiohttp.ClientRequest, aiohttp.ClientResponse, Mapping[str, Any], Optional[Mapping[str, Any]]], Iterable[StreamData] + ], + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> Iterable[StreamData]: + self._session = await self._create_session() + assert self._session + assert not self._session.closed + try: + stream_state = stream_state or {} + pagination_complete = False + next_page_token = None + while not pagination_complete: + async def f(): + nonlocal next_page_token + assert not self._session.closed + request, response = await self._fetch_next_page(stream_slice, stream_state, next_page_token) + next_page_token = await self.next_page_token(response) + return request, response, next_page_token + + request, response, next_page_token = await f() + + async for record in records_generator_fn(request, response, stream_state, stream_slice): + yield record + + if not next_page_token: + pagination_complete = True + finally: + await self._session.close() + + async def _fetch_next_page( + self, + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Tuple[aiohttp.ClientRequest, aiohttp.ClientResponse]: # TODO: maybe don't need to return request too since its on aiohttp.ClientResponse + request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + request = self._create_prepared_request( + path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + headers=dict(request_headers, **self.authenticator.get_auth_header()), + params=self.request_params(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + json=self.request_body_json(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + data=self.request_body_data(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + ) + request_kwargs = self.request_kwargs(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + + assert not self._session.closed + response = await self._send_request(request, request_kwargs) + return request, response + + +class AsyncHttpSubStream(AsyncHttpStream, ABC): + def __init__(self, parent: AsyncHttpStream, **kwargs: Any): + """ + :param parent: should be the instance of HttpStream class + """ + super().__init__(**kwargs) + self.parent = parent + + async def stream_slices( + self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None + ) -> Iterable[Optional[Mapping[str, Any]]]: + parent_stream_slices = self.parent.stream_slices( + sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_state=stream_state + ) + + # iterate over all parent stream_slices + async for stream_slice in parent_stream_slices: + parent_records = self.parent.read_records( + sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_slice=stream_slice, stream_state=stream_state + ) + + # iterate over all parent records with current stream_slice + for record in parent_records: + yield {"parent": record} From 8793464e0818a5aa1c0e01a44c9c93911271ea04 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 21 Dec 2023 21:56:43 -0500 Subject: [PATCH 02/74] Working: new file for async exceptions (some overlap with existing) --- .../sources/streams/http/async_exceptions.py | 41 +++++++++++++++++++ .../sources/streams/http/async_http.py | 2 +- 2 files changed, 42 insertions(+), 1 deletion(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_exceptions.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_exceptions.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_exceptions.py new file mode 100644 index 000000000000..ea5228fab2b5 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_exceptions.py @@ -0,0 +1,41 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + + +from typing import Union + +import aiohttp + + +class BaseBackoffException(aiohttp.ClientResponseError): + def __init__(self, response: aiohttp.ClientResponse, error_message: str = ""): + error_message = ( + error_message or f"Request URL: {response.request_info.url}, Response Code: {response.status}, Response Text: {response.text}" + ) + super().__init__(request_info=response.request_info, history=(response,), status=response.status, message=error_message, headers=response.headers) + + +class RequestBodyException(Exception): + """ + Raised when there are issues in configuring a request body + """ + + +class UserDefinedBackoffException(BaseBackoffException): + """ + An exception that exposes how long it attempted to backoff + """ + + def __init__(self, backoff: Union[int, float], response: aiohttp.ClientResponse, error_message: str = ""): + """ + :param backoff: how long to backoff in seconds + :param request: the request that triggered this backoff exception + :param response: the response that triggered the backoff exception + """ + self.backoff = backoff + super().__init__(response=response, error_message=error_message) + + +class DefaultBackoffException(BaseBackoffException): + pass diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py index 6ebe8c978706..3465a3cd7628 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py @@ -19,11 +19,11 @@ from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http import HttpStream +from airbyte_cdk.sources.streams.http.async_exceptions import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH from .auth.core import HttpAuthenticator, NoAuth -from .exceptions import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from .rate_limiting import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies From 3e39da764355cbc18d5800394d6190feb2945d41 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 21 Dec 2023 22:06:17 -0500 Subject: [PATCH 03/74] file rename --- .../streams/http/{async_exceptions.py => exceptions_async.py} | 0 .../sources/streams/http/{async_http.py => http_async.py} | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename airbyte-cdk/python/airbyte_cdk/sources/streams/http/{async_exceptions.py => exceptions_async.py} (100%) rename airbyte-cdk/python/airbyte_cdk/sources/streams/http/{async_http.py => http_async.py} (99%) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_exceptions.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions_async.py similarity index 100% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_exceptions.py rename to airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py similarity index 99% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py rename to airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 3465a3cd7628..be0d2816a0f4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/async_http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -19,7 +19,7 @@ from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http import HttpStream -from airbyte_cdk.sources.streams.http.async_exceptions import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException +from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH From e979302ad853aa1e184e320c6bb0672c6bfaddf2 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 21 Dec 2023 22:13:34 -0500 Subject: [PATCH 04/74] Working: new core_async.py with AsyncStream --- .../airbyte_cdk/sources/streams/core_async.py | 114 ++++++++++++++++++ .../sources/streams/http/http_async.py | 3 +- 2 files changed, 116 insertions(+), 1 deletion(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py new file mode 100644 index 000000000000..881e2809383a --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -0,0 +1,114 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + + +import logging +from abc import ABC, abstractmethod +from typing import Any, Iterable, List, Mapping, MutableMapping, Optional + +from airbyte_cdk.models import SyncMode +from airbyte_cdk.models import Type as MessageType +from airbyte_cdk.sources.streams.core import Stream, StreamData +from airbyte_cdk.sources.utils.schema_helpers import InternalConfig +from airbyte_cdk.sources.utils.slice_logger import SliceLogger + + +class AsyncStream(Stream, ABC): + """ + Base abstract class for an Airbyte Stream. Makes no assumption of the Stream's underlying transport protocol. + """ + + async def read_full_refresh( + self, + cursor_field: Optional[List[str]], + logger: logging.Logger, + slice_logger: SliceLogger, + ) -> Iterable[StreamData]: + slices = self.stream_slices(sync_mode=SyncMode.full_refresh, cursor_field=cursor_field) + logger.debug(f"Processing stream slices for {self.name} (sync_mode: full_refresh)", extra={"stream_slices": slices}) + for _slice in slices: + if slice_logger.should_log_slice_message(logger): + yield slice_logger.create_slice_log_message(_slice) + async for record in self.read_records( + stream_slice=_slice, + sync_mode=SyncMode.full_refresh, + cursor_field=cursor_field, + ): + yield record + + async def read_incremental( # type: ignore # ignoring typing for ConnectorStateManager because of circular dependencies + self, + cursor_field: Optional[List[str]], + logger: logging.Logger, + slice_logger: SliceLogger, + stream_state: MutableMapping[str, Any], + state_manager, + per_stream_state_enabled: bool, + internal_config: InternalConfig, + ) -> Iterable[StreamData]: + slices = self.stream_slices( + cursor_field=cursor_field, + sync_mode=SyncMode.incremental, + stream_state=stream_state, + ) + logger.debug(f"Processing stream slices for {self.name} (sync_mode: incremental)", extra={"stream_slices": slices}) + + has_slices = False + record_counter = 0 + for _slice in slices: + has_slices = True + if slice_logger.should_log_slice_message(logger): + yield slice_logger.create_slice_log_message(_slice) + records = self.read_records( + sync_mode=SyncMode.incremental, + stream_slice=_slice, + stream_state=stream_state, + cursor_field=cursor_field or None, + ) + for record_data_or_message in records: + yield record_data_or_message + if isinstance(record_data_or_message, Mapping) or ( + hasattr(record_data_or_message, "type") and record_data_or_message.type == MessageType.RECORD + ): + record_data = record_data_or_message if isinstance(record_data_or_message, Mapping) else record_data_or_message.record + stream_state = self.get_updated_state(stream_state, record_data) + checkpoint_interval = self.state_checkpoint_interval + record_counter += 1 + if checkpoint_interval and record_counter % checkpoint_interval == 0: + yield self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) + + if internal_config.is_limit_reached(record_counter): + break + + yield self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) + + if not has_slices: + # Safety net to ensure we always emit at least one state message even if there are no slices + checkpoint = self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) + yield checkpoint + + @abstractmethod + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> Iterable[StreamData]: + """ + This method should be overridden by subclasses to read records based on the inputs + """ + + async def stream_slices( + self, *, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None + ) -> Iterable[Optional[Mapping[str, Any]]]: + """ + Override to define the slices for this stream. See the stream slicing section of the docs for more information. + + :param sync_mode: + :param cursor_field: + :param stream_state: + :return: + """ + return [None] diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index be0d2816a0f4..1e8b04d7d96f 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -18,6 +18,7 @@ from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.streams.core_async import AsyncStream from airbyte_cdk.sources.streams.http import HttpStream from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.utils.types import JsonType @@ -31,7 +32,7 @@ T = TypeVar("T") -class AsyncHttpStream(HttpStream, ABC): +class AsyncHttpStream(HttpStream, AsyncStream, ABC): """ Base abstract class for an Airbyte Stream using the HTTP protocol with asyncio. From bb5f26b9b0c47b21a7ef8f5dce2acdc4bb589e9d Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 21 Dec 2023 22:19:28 -0500 Subject: [PATCH 05/74] Working: stream_helper_async; not sure if needed --- .../streams/utils/stream_helper_async.py | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py new file mode 100644 index 000000000000..6f9deb3b6cf8 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py @@ -0,0 +1,40 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +from typing import Any, Mapping, Optional + +from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.streams.core import Stream, StreamData + + +def get_first_stream_slice(stream) -> Optional[Mapping[str, Any]]: + """ + Gets the first stream_slice from a given stream's stream_slices. + :param stream: stream + :raises StopIteration: if there is no first slice to return (the stream_slices generator is empty) + :return: first stream slice from 'stream_slices' generator (`None` is a valid stream slice) + """ + # We wrap the return output of stream_slices() because some implementations return types that are iterable, + # but not iterators such as lists or tuples + slices = iter( + stream.stream_slices( + cursor_field=stream.cursor_field, + sync_mode=SyncMode.full_refresh, + ) + ) + return anext(slices) + + +def get_first_record_for_slice(stream: Stream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: + """ + Gets the first record for a stream_slice of a stream. + :param stream: stream + :param stream_slice: stream_slice + :raises StopIteration: if there is no first record to return (the read_records generator is empty) + :return: StreamData containing the first record in the slice + """ + # We wrap the return output of read_records() because some implementations return types that are iterable, + # but not iterators such as lists or tuples + records_for_slice = stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slice) + return anext(records_for_slice) From ee29b916dbfe9a28782931b58bfd7378b53e9636 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 20:39:07 -0500 Subject: [PATCH 06/74] Working: AsyncAbstractSource --- .../sources/abstract_source_async.py | 297 ++++++++++++++++++ 1 file changed, 297 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py new file mode 100644 index 000000000000..67bdc358e1f6 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -0,0 +1,297 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import asyncio +import logging +from abc import ABC, abstractmethod +from queue import Queue +from threading import Thread +from typing import Any, Callable, Dict, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, TypeVar, Union + +from airbyte_cdk.models import ( + AirbyteMessage, + AirbyteStateMessage, + AirbyteStreamStatus, + ConfiguredAirbyteCatalog, + ConfiguredAirbyteStream, + SyncMode, +) +from airbyte_cdk.models import Type as MessageType +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.abstract_source import AbstractSource +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.streams.http.http import HttpStream +from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message +from airbyte_cdk.sources.utils.schema_helpers import InternalConfig, split_config +from airbyte_cdk.utils.event_timing import create_timer +from airbyte_cdk.utils.stream_status_utils import as_airbyte_message as stream_status_as_airbyte_message +from airbyte_cdk.utils.traced_exception import AirbyteTracedException + +T = TypeVar("T") + + +class Sentinel: + def __init__(self, name: str): + self.name = name + + +class SourceReader(Iterator): + def __init__(self, queue: Queue, sentinels: Dict[str, Sentinel], reader_fn: Callable, *args: Any): + self.queue = queue + self.sentinels = sentinels + self.reader_fn = reader_fn + self.reader_args = args + + self.thread = Thread(target=self._start_reader_thread) + self.thread.start() + + def _start_reader_thread(self): + asyncio.run(self.reader_fn(*self.reader_args)) + + def __next__(self): + item = self.queue.get() + if isinstance(item, Sentinel): + print(f">>>>>>>>> Finished {item.name} stream") + self.sentinels.pop(item.name) # TODO: error handling? + if not self.sentinels: + self.thread.join() + raise StopIteration + else: + return self.__next__() + else: + return item + + +class AsyncAbstractSource(AbstractSource, ABC): + """ + Abstract base class for an Airbyte Source. Consumers should implement any abstract methods + in this class to create an Airbyte Specification compliant Source. + """ + + def __init__(self): + super().__init__() + self.queue = Queue(10000) + + @abstractmethod + def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + """ + :param logger: 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. + """ + + @abstractmethod + def streams(self, config: Mapping[str, Any]) -> List[Stream]: + """ + :param config: The user-provided configuration as specified by the source's spec. + Any stream construction related operation should happen here. + :return: A list of the streams in this source connector. + """ + + def read( + self, + logger: logging.Logger, + config: Mapping[str, Any], + catalog: ConfiguredAirbyteCatalog, + state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + ) -> Iterator[AirbyteMessage]: + """Implements the Read operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/.""" + logger.info(f"Starting syncing {self.name}") + config, internal_config = split_config(config) + # TODO assert all streams exist in the connector + # get the streams once in case the connector needs to make any queries to generate them + stream_instances = {s.name: s for s in self.streams(config)} + state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) + self._stream_to_instance_map = stream_instances + + n_records = 0 + with create_timer(self.name) as timer: + for record in self._do_read(catalog, stream_instances, timer, logger, state_manager, internal_config): + n_records += 1 + yield record + + print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") + logger.info(f"Finished syncing {self.name}") + + def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + streams_in_progress = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams} + for record in SourceReader(self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config): + yield record + + async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + tasks = [asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config)) for s in catalog.streams] + await asyncio.gather(*tasks) + + async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + print(f">>>>>>>>> _do_async_read_stream {configured_stream.stream.name}") + try: + await self._async_read_stream(configured_stream, stream_instances, timer, logger, state_manager, internal_config) + finally: + self.queue.put(Sentinel(configured_stream.stream.name)) + + async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + stream_instance = stream_instances.get("Account") + if not stream_instance: + if not self.raise_exception_on_missing_stream: + return + # raise KeyError( + # f"The stream {configured_stream.stream.name} no longer exists in the configuration. " + # f"Refresh the schema in replication settings and remove this stream from future sync attempts." + # ) + + try: + # timer.start_event(f"Syncing stream {configured_stream.stream.name}") + # stream_is_available, reason = stream_instance.check_availability(logger, self) + # if not stream_is_available: + # logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") + # return + logger.info(f"Marking stream {configured_stream.stream.name} as STARTED") + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.STARTED)) + async for record in self._read_stream( + logger=logger, + stream_instance=stream_instance, + configured_stream=configured_stream, + state_manager=state_manager, + internal_config=internal_config, + ): + self.queue.put(record) + logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) + except AirbyteTracedException as e: + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + raise e + except Exception as e: + for message in self._emit_queued_messages(): + self.queue.put(message) + logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") + logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + display_message = stream_instance.get_error_display_message(e) + if display_message: + raise AirbyteTracedException.from_exception(e, message=display_message) from e + raise e + finally: + # timer.finish_event() + logger.info(f"Finished syncing {configured_stream.stream.name}") + # logger.info(timer.report()) + + async def _read_stream( + self, + logger: logging.Logger, + stream_instance: Stream, + configured_stream: ConfiguredAirbyteStream, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ) -> Iterator[AirbyteMessage]: + if internal_config.page_size and isinstance(stream_instance, HttpStream): + logger.info(f"Setting page size for {stream_instance.name} to {internal_config.page_size}") + stream_instance.page_size = internal_config.page_size + logger.debug( + f"Syncing configured stream: {configured_stream.stream.name}", + extra={ + "sync_mode": configured_stream.sync_mode, + "primary_key": configured_stream.primary_key, + "cursor_field": configured_stream.cursor_field, + }, + ) + stream_instance.log_stream_sync_configuration() + + use_incremental = configured_stream.sync_mode == SyncMode.incremental and stream_instance.supports_incremental + if use_incremental: + record_iterator = self._read_incremental( + logger, + stream_instance, + configured_stream, + state_manager, + internal_config, + ) + else: + record_iterator = self._read_full_refresh(logger, stream_instance, configured_stream, internal_config) + + record_counter = 0 + stream_name = configured_stream.stream.name + logger.info(f"Syncing stream: {stream_name} ") + async for record in record_iterator: + if record.type == MessageType.RECORD: + record_counter += 1 + if record_counter == 1: + logger.info(f"Marking stream {stream_name} as RUNNING") + # If we just read the first record of the stream, emit the transition to the RUNNING state + yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.RUNNING) + for message in self._emit_queued_messages(): + yield message + yield record + + logger.info(f"Read {record_counter} records from {stream_name} stream") + + async def _read_incremental( + self, + logger: logging.Logger, + stream_instance: Stream, + configured_stream: ConfiguredAirbyteStream, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ) -> Iterator[AirbyteMessage]: + """Read stream using incremental algorithm + + :param logger: + :param stream_instance: + :param configured_stream: + :param state_manager: + :param internal_config: + :return: + """ + stream_name = configured_stream.stream.name + stream_state = state_manager.get_stream_state(stream_name, stream_instance.namespace) + + if stream_state and "state" in dir(stream_instance): + stream_instance.state = stream_state # type: ignore # we check that state in the dir(stream_instance) + logger.info(f"Setting state of {self.name} stream to {stream_state}") + + async for record_data_or_message in stream_instance.read_incremental( + configured_stream.cursor_field, + logger, + self._slice_logger, + stream_state, + state_manager, + self.per_stream_state_enabled, + internal_config, + ): + yield self._get_message(record_data_or_message, stream_instance) + + def _emit_queued_messages(self) -> Iterable[AirbyteMessage]: + if self.message_repository: + yield from self.message_repository.consume_queue() + return + + async def _read_full_refresh( + self, + logger: logging.Logger, + stream_instance: Stream, + configured_stream: ConfiguredAirbyteStream, + internal_config: InternalConfig, + ) -> Iterator[AirbyteMessage]: + total_records_counter = 0 + async for record_data_or_message in stream_instance.read_full_refresh(configured_stream.cursor_field, logger, self._slice_logger): + message = self._get_message(record_data_or_message, stream_instance) + yield message + if message.type == MessageType.RECORD: + total_records_counter += 1 + if internal_config.is_limit_reached(total_records_counter): + return + + def _get_message(self, record_data_or_message: Union[StreamData, AirbyteMessage], stream: Stream) -> AirbyteMessage: + """ + Converts the input to an AirbyteMessage if it is a StreamData. Returns the input as is if it is already an AirbyteMessage + """ + if isinstance(record_data_or_message, AirbyteMessage): + return record_data_or_message + else: + return stream_data_to_airbyte_message(stream.name, record_data_or_message, stream.transformer, stream.get_json_schema()) From d138bcf89466f2d55510149d06db1db874a435b2 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 20:51:19 -0500 Subject: [PATCH 07/74] Working: AsyncAbstractStream --- .../concurrent/abstract_stream_async.py | 83 +++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py new file mode 100644 index 000000000000..4f1281d37cdd --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py @@ -0,0 +1,83 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +from abc import ABC, abstractmethod +from typing import Any, Iterable, Mapping, Optional + +from airbyte_cdk.models import AirbyteStream +from airbyte_cdk.sources.streams.concurrent.availability_strategy import StreamAvailability +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from deprecated.classic import deprecated + + +@deprecated("This class is experimental. Use at your own risk.") +class AsyncAbstractStream(ABC): + """ + AbstractStream is an experimental interface for streams developed as part of the Concurrent CDK. + This interface is not yet stable and may change in the future. Use at your own risk. + + Why create a new interface instead of adding concurrency capabilities the existing Stream? + We learnt a lot since the initial design of the Stream interface, and we wanted to take the opportunity to improve. + + High level, the changes we are targeting are: + - Removing superfluous or leaky parameters from the methods' interfaces + - Using composition instead of inheritance to add new capabilities + + To allow us to iterate fast while ensuring backwards compatibility, we are creating a new interface with a facade object that will bridge the old and the new interfaces. + Source connectors that wish to leverage concurrency need to implement this new interface. An example will be available shortly + + Current restrictions on sources that implement this interface. Not all of these restrictions will be lifted in the future, but most will as we iterate on the design. + - Only full refresh is supported. This will be addressed in the future. + - The read method does not accept a cursor_field. Streams must be internally aware of the cursor field to use. User-defined cursor fields can be implemented by modifying the connector's main method to instantiate the streams with the configured cursor field. + - Streams cannot return user-friendly messages by overriding Stream.get_error_display_message. This will be addressed in the future. + - The Stream's behavior cannot depend on a namespace + - TypeTransformer is not supported. This will be addressed in the future. + - Nested cursor and primary keys are not supported + """ + + @abstractmethod + async def generate_partitions(self) -> Iterable[Partition]: + """ + Generates the partitions that will be read by this stream. + :return: An iterable of partitions. + """ + + @property + @abstractmethod + def name(self) -> str: + """ + :return: The stream name + """ + + @property + @abstractmethod + def cursor_field(self) -> Optional[str]: + """ + Override to return the default cursor field used by this stream e.g: an API entity might always use created_at as the cursor field. + :return: The name of the field used as a cursor. Nested cursor fields are not supported. + """ + + @abstractmethod + def check_availability(self) -> StreamAvailability: + """ + :return: The stream's availability + """ + + @abstractmethod + def get_json_schema(self) -> Mapping[str, Any]: + """ + :return: A dict of the JSON schema representing this stream. + """ + + @abstractmethod + def as_airbyte_stream(self) -> AirbyteStream: + """ + :return: A dict of the JSON schema representing this stream. + """ + + @abstractmethod + def log_stream_sync_configuration(self) -> None: + """ + Logs the stream's configuration for debugging purposes. + """ From 14fe32bd3ad595db7900e46e6ffd29c9a73e1ca3 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 20:58:05 -0500 Subject: [PATCH 08/74] Working: AsyncDefaultStream --- .../concurrent/default_stream_async.py | 80 +++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py new file mode 100644 index 000000000000..86b86f033a1e --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py @@ -0,0 +1,80 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +from functools import lru_cache +from logging import Logger +from typing import Any, Iterable, List, Mapping, Optional + +from airbyte_cdk.models import AirbyteStream, SyncMode +from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream +from airbyte_cdk.sources.streams.concurrent.availability_strategy import AbstractAvailabilityStrategy, StreamAvailability +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator + + +class AsyncDefaultStream(AbstractStream): + def __init__( + self, + partition_generator: PartitionGenerator, + name: str, + json_schema: Mapping[str, Any], + availability_strategy: AbstractAvailabilityStrategy, + primary_key: List[str], + cursor_field: Optional[str], + logger: Logger, + namespace: Optional[str] = None, + ) -> None: + self._stream_partition_generator = partition_generator + self._name = name + self._json_schema = json_schema + self._availability_strategy = availability_strategy + self._primary_key = primary_key + self._cursor_field = cursor_field + self._logger = logger + self._namespace = namespace + + async def generate_partitions(self) -> Iterable[Partition]: + async for partition in self._stream_partition_generator.generate(): + yield partition + + @property + def name(self) -> str: + return self._name + + def check_availability(self) -> StreamAvailability: + return self._availability_strategy.check_availability(self._logger) + + @property + def cursor_field(self) -> Optional[str]: + return self._cursor_field + + @lru_cache(maxsize=None) + def get_json_schema(self) -> Mapping[str, Any]: + return self._json_schema + + def as_airbyte_stream(self) -> AirbyteStream: + stream = AirbyteStream(name=self.name, json_schema=dict(self._json_schema), supported_sync_modes=[SyncMode.full_refresh]) + + if self._namespace: + stream.namespace = self._namespace + + if self._cursor_field: + stream.source_defined_cursor = True + stream.supported_sync_modes.append(SyncMode.incremental) + stream.default_cursor_field = [self._cursor_field] + + keys = self._primary_key + if keys and len(keys) > 0: + stream.source_defined_primary_key = [keys] + + return stream + + def log_stream_sync_configuration(self) -> None: + self._logger.debug( + f"Syncing stream instance: {self.name}", + extra={ + "primary_key": self._primary_key, + "cursor_field": self.cursor_field, + }, + ) From 6fb1eea460b0302f2f938d5354c5b5a911cc44fa Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:00:23 -0500 Subject: [PATCH 09/74] Working: AsyncPartitionGenerator; not used --- .../partitions/partition_generator_async.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py new file mode 100644 index 000000000000..d64663d927a1 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py @@ -0,0 +1,18 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +from abc import ABC, abstractmethod +from typing import Iterable + +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition + + +class AsyncPartitionGenerator(ABC): + @abstractmethod + async def generate(self) -> Iterable[Partition]: + """ + Generates partitions for a given sync mode. + :return: An iterable of partitions + """ + pass From 65e82418419502b345170d1fd68d8dbc64b57df5 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:02:02 -0500 Subject: [PATCH 10/74] Not used: async_call_rate and rate_limiting_async --- .../sources/streams/async_call_rate.py | 500 ++++++++++++++++++ .../streams/http/rate_limiting_async.py | 100 ++++ 2 files changed, 600 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py new file mode 100644 index 000000000000..edb024724736 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py @@ -0,0 +1,500 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import abc +import dataclasses +import datetime +import logging +import time +from datetime import timedelta +from threading import RLock +from typing import TYPE_CHECKING, Any, Mapping, Optional +from urllib import parse + +import aiohttp +import aiohttp_client_cache +from pyrate_limiter import InMemoryBucket, Limiter +from pyrate_limiter import Rate as PyRateRate +from pyrate_limiter import RateItem, TimeClock +from pyrate_limiter.exceptions import BucketFullException + +from airbyte_cdk.sources.streams.call_rate import CallRateLimitHit, Rate + +# prevents mypy from complaining about missing session attributes in LimiterMixin +if TYPE_CHECKING: + MIXIN_BASE = aiohttp.ClientSession +else: + MIXIN_BASE = object + +logger = logging.getLogger("airbyte") + + +class AbstractCallRatePolicy(abc.ABC): + """Call rate policy interface. + Should be configurable with different rules, like N per M for endpoint X. Endpoint X is matched with APIBudget. + """ + + @abc.abstractmethod + def matches(self, request: Any) -> bool: + """Tells if this policy matches specific request and should apply to it + + :param request: + :return: True if policy should apply to this request, False - otherwise + """ + + @abc.abstractmethod + def try_acquire(self, request: Any, weight: int) -> None: + """Try to acquire request + + :param request: a request object representing a single call to API + :param weight: number of requests to deduct from credit + :return: + """ + + @abc.abstractmethod + def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: + """Update call rate counting with current values + + :param available_calls: + :param call_reset_ts: + """ + + +class RequestMatcher(abc.ABC): + """Callable that help to match a request object with call rate policies.""" + + @abc.abstractmethod + def __call__(self, request: Any) -> bool: + """ + + :param request: + :return: True if matches the provided request object, False - otherwise + """ + + +class HttpRequestMatcher(RequestMatcher): + """Simple implementation of RequestMatcher for http requests case""" + + def __init__( + self, + method: Optional[str] = None, + url: Optional[str] = None, + params: Optional[Mapping[str, Any]] = None, + headers: Optional[Mapping[str, Any]] = None, + ): + """Constructor + + :param method: + :param url: + :param params: + :param headers: + """ + self._method = method + self._url = url + self._params = {str(k): str(v) for k, v in (params or {}).items()} + self._headers = {str(k): str(v) for k, v in (headers or {}).items()} + + @staticmethod + def _match_dict(obj: Mapping[str, Any], pattern: Mapping[str, Any]) -> bool: + """Check that all elements from pattern dict present and have the same values in obj dict + + :param obj: + :param pattern: + :return: + """ + return pattern.items() <= obj.items() + + def __call__(self, request: Any) -> bool: + """ + + :param request: + :return: True if matches the provided request object, False - otherwise + """ + # if isinstance(request, requests.Request): # TODO: do I need to handle a non-"prepared" request? + # prepared_request = request.prepare() + if isinstance(request, aiohttp.ClientRequest): + prepared_request = request + else: + return False + + if self._method is not None: + if prepared_request.method != self._method: + return False + if self._url is not None and prepared_request.url is not None: + url_without_params = prepared_request.url.split("?")[0] + if url_without_params != self._url: + return False + if self._params is not None: + parsed_url = parse.urlsplit(prepared_request.url) + params = dict(parse.parse_qsl(str(parsed_url.query))) + if not self._match_dict(params, self._params): + return False + if self._headers is not None: + if not self._match_dict(prepared_request.headers, self._headers): + return False + return True + + +class BaseCallRatePolicy(AbstractCallRatePolicy, abc.ABC): + def __init__(self, matchers: list[RequestMatcher]): + self._matchers = matchers + + def matches(self, request: Any) -> bool: + """Tell if this policy matches specific request and should apply to it + + :param request: + :return: True if policy should apply to this request, False - otherwise + """ + + if not self._matchers: + return True + return any(matcher(request) for matcher in self._matchers) + + +class UnlimitedCallRatePolicy(BaseCallRatePolicy): + """ + This policy is for explicit unlimited call rates. + It can be used when we want to match a specific group of requests and don't apply any limits. + + Example: + + APICallBudget( + [ + UnlimitedCallRatePolicy( + matchers=[HttpRequestMatcher(url="/some/method", headers={"sandbox": true})], + ), + FixedWindowCallRatePolicy( + matchers=[HttpRequestMatcher(url="/some/method")], + next_reset_ts=datetime.now(), + period=timedelta(hours=1) + call_limit=1000, + ), + ] + ) + + The code above will limit all calls to /some/method except calls that have header sandbox=True + """ + + def try_acquire(self, request: Any, weight: int) -> None: + """Do nothing""" + + def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: + """Do nothing""" + + +class FixedWindowCallRatePolicy(BaseCallRatePolicy): + def __init__(self, next_reset_ts: datetime.datetime, period: timedelta, call_limit: int, matchers: list[RequestMatcher]): + """A policy that allows {call_limit} calls within a {period} time interval + + :param next_reset_ts: next call rate reset time point + :param period: call rate reset period + :param call_limit: + :param matchers: + """ + + self._next_reset_ts = next_reset_ts + self._offset = period + self._call_limit = call_limit + self._calls_num = 0 + self._lock = RLock() + super().__init__(matchers=matchers) + + def try_acquire(self, request: Any, weight: int) -> None: + if weight > self._call_limit: + raise ValueError("Weight can not exceed the call limit") + if not self.matches(request): + raise ValueError("Request does not match the policy") + + with self._lock: + self._update_current_window() + + if self._calls_num + weight > self._call_limit: + reset_in = self._next_reset_ts - datetime.datetime.now() + error_message = ( + f"reached maximum number of allowed calls {self._call_limit} " f"per {self._offset} interval, next reset in {reset_in}." + ) + raise CallRateLimitHit( + error=error_message, + item=request, + weight=weight, + rate=f"{self._call_limit} per {self._offset}", + time_to_wait=reset_in, + ) + + self._calls_num += weight + + def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: + """Update call rate counters, by default, only reacts to decreasing updates of available_calls and changes to call_reset_ts. + We ignore updates with available_calls > current_available_calls to support call rate limits that are lower than API limits. + + :param available_calls: + :param call_reset_ts: + """ + with self._lock: + self._update_current_window() + current_available_calls = self._call_limit - self._calls_num + + if available_calls is not None and current_available_calls > available_calls: + logger.debug( + "got rate limit update from api, adjusting available calls from %s to %s", current_available_calls, available_calls + ) + self._calls_num = self._call_limit - available_calls + + if call_reset_ts is not None and call_reset_ts != self._next_reset_ts: + logger.debug("got rate limit update from api, adjusting reset time from %s to %s", self._next_reset_ts, call_reset_ts) + self._next_reset_ts = call_reset_ts + + def _update_current_window(self) -> None: + now = datetime.datetime.now() + if now > self._next_reset_ts: + logger.debug("started new window, %s calls available now", self._call_limit) + self._next_reset_ts = self._next_reset_ts + self._offset + self._calls_num = 0 + + +class MovingWindowCallRatePolicy(BaseCallRatePolicy): + """ + Policy to control requests rate implemented on top of PyRateLimiter lib. + The main difference between this policy and FixedWindowCallRatePolicy is that the rate-limiting window + is moving along requests that we made, and there is no moment when we reset an available number of calls. + This strategy requires saving of timestamps of all requests within a window. + """ + + def __init__(self, rates: list[Rate], matchers: list[RequestMatcher]): + """Constructor + + :param rates: list of rates, the order is important and must be ascending + :param matchers: + """ + if not rates: + raise ValueError("The list of rates can not be empty") + pyrate_rates = [PyRateRate(limit=rate.limit, interval=int(rate.interval.total_seconds() * 1000)) for rate in rates] + self._bucket = InMemoryBucket(pyrate_rates) + # Limiter will create the background task that clears old requests in the bucket + self._limiter = Limiter(self._bucket) + super().__init__(matchers=matchers) + + def try_acquire(self, request: Any, weight: int) -> None: + if not self.matches(request): + raise ValueError("Request does not match the policy") + + try: + self._limiter.try_acquire(request, weight=weight) + except BucketFullException as exc: + item = self._limiter.bucket_factory.wrap_item(request, weight) + assert isinstance(item, RateItem) + + with self._limiter.lock: + time_to_wait = self._bucket.waiting(item) + assert isinstance(time_to_wait, int) + + raise CallRateLimitHit( + error=str(exc.meta_info["error"]), + item=request, + weight=int(exc.meta_info["weight"]), + rate=str(exc.meta_info["rate"]), + time_to_wait=timedelta(milliseconds=time_to_wait), + ) + + def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: + """Adjust call bucket to reflect the state of the API server + + :param available_calls: + :param call_reset_ts: + :return: + """ + if available_calls is not None and call_reset_ts is None: # we do our best to sync buckets with API + if available_calls == 0: + with self._limiter.lock: + items_to_add = self._bucket.count() < self._bucket.rates[0].limit + if items_to_add > 0: + now: int = TimeClock().now() # type: ignore[no-untyped-call] + self._bucket.put(RateItem(name="dummy", timestamp=now, weight=items_to_add)) + # TODO: add support if needed, it might be that it is not possible to make a good solution for this case + # if available_calls is not None and call_reset_ts is not None: + # ts = call_reset_ts.timestamp() + + +class AbstractAPIBudget(abc.ABC): + """Interface to some API where a client allowed to have N calls per T interval. + + Important: APIBudget is not doing any API calls, the end user code is responsible to call this interface + to respect call rate limitation of the API. + + It supports multiple policies applied to different group of requests. To distinct these groups we use RequestMatchers. + Individual policy represented by MovingWindowCallRatePolicy and currently supports only moving window strategy. + """ + + @abc.abstractmethod + def acquire_call(self, request: Any, block: bool = True, timeout: Optional[float] = None) -> None: + """Try to get a call from budget, will block by default + + :param request: + :param block: when true (default) will block the current thread until call credit is available + :param timeout: if set will limit maximum time in block, otherwise will wait until credit is available + :raises: CallRateLimitHit - when no credits left and if timeout was set the waiting time exceed the timeout + """ + + @abc.abstractmethod + def get_matching_policy(self, request: Any) -> Optional[AbstractCallRatePolicy]: + """Find matching call rate policy for specific request""" + + @abc.abstractmethod + def update_from_response(self, request: Any, response: Any) -> None: + """Update budget information based on response from API + + :param request: the initial request that triggered this response + :param response: response from the API + """ + + +class APIBudget(AbstractAPIBudget): + """Default APIBudget implementation""" + + def __init__(self, policies: list[AbstractCallRatePolicy], maximum_attempts_to_acquire: int = 100000) -> None: + """Constructor + + :param policies: list of policies in this budget + :param maximum_attempts_to_acquire: number of attempts before throwing hit ratelimit exception, we put some big number here + to avoid situations when many threads compete with each other for a few lots over a significant amount of time + """ + + self._policies = policies + self._maximum_attempts_to_acquire = maximum_attempts_to_acquire + + def get_matching_policy(self, request: Any) -> Optional[AbstractCallRatePolicy]: + for policy in self._policies: + if policy.matches(request): + return policy + return None + + def acquire_call(self, request: Any, block: bool = True, timeout: Optional[float] = None) -> None: + """Try to get a call from budget, will block by default. + Matchers will be called sequentially in the same order they were added. + The first matcher that returns True will + + :param request: + :param block: when true (default) will block the current thread until call credit is available + :param timeout: if provided will limit maximum time in block, otherwise will wait until credit is available + :raises: CallRateLimitHit - when no calls left and if timeout was set the waiting time exceed the timeout + """ + + policy = self.get_matching_policy(request) + if policy: + self._do_acquire(request=request, policy=policy, block=block, timeout=timeout) + elif self._policies: + logger.info("no policies matched with requests, allow call by default") + + def update_from_response(self, request: Any, response: Any) -> None: + """Update budget information based on response from API + + :param request: the initial request that triggered this response + :param response: response from the API + """ + pass + + def _do_acquire(self, request: Any, policy: AbstractCallRatePolicy, block: bool, timeout: Optional[float]) -> None: + """Internal method to try to acquire a call credit + + :param request: + :param policy: + :param block: + :param timeout: + """ + last_exception = None + # sometimes we spend all budget before a second attempt, so we have few more here + for attempt in range(1, self._maximum_attempts_to_acquire): + try: + policy.try_acquire(request, weight=1) + return + except CallRateLimitHit as exc: + last_exception = exc + if block: + if timeout is not None: + time_to_wait = min(timedelta(seconds=timeout), exc.time_to_wait) + else: + time_to_wait = exc.time_to_wait + + time_to_wait = max(timedelta(0), time_to_wait) # sometimes we get negative duration + logger.info("reached call limit %s. going to sleep for %s", exc.rate, time_to_wait) + time.sleep(time_to_wait.total_seconds()) + else: + raise + + if last_exception: + logger.info("we used all %s attempts to acquire and failed", self._maximum_attempts_to_acquire) + raise last_exception + + +class HttpAPIBudget(APIBudget): + """Implementation of AbstractAPIBudget for HTTP""" + + def __init__( + self, + ratelimit_reset_header: str = "ratelimit-reset", + ratelimit_remaining_header: str = "ratelimit-remaining", + status_codes_for_ratelimit_hit: tuple[int] = (429,), + **kwargs: Any, + ): + """Constructor + + :param ratelimit_reset_header: name of the header that has a timestamp of the next reset of call budget + :param ratelimit_remaining_header: name of the header that has the number of calls left + :param status_codes_for_ratelimit_hit: list of HTTP status codes that signal about rate limit being hit + """ + self._ratelimit_reset_header = ratelimit_reset_header + self._ratelimit_remaining_header = ratelimit_remaining_header + self._status_codes_for_ratelimit_hit = status_codes_for_ratelimit_hit + super().__init__(**kwargs) + + def update_from_response(self, request: Any, response: Any) -> None: + policy = self.get_matching_policy(request) + if not policy: + return + + if isinstance(response, aiohttp.ClientResponse): + available_calls = self.get_calls_left_from_response(response) + reset_ts = self.get_reset_ts_from_response(response) + policy.update(available_calls=available_calls, call_reset_ts=reset_ts) + + def get_reset_ts_from_response(self, response: aiohttp.ClientResponse) -> Optional[datetime.datetime]: + if response.headers.get(self._ratelimit_reset_header): + return datetime.datetime.fromtimestamp(int(response.headers[self._ratelimit_reset_header])) + return None + + def get_calls_left_from_response(self, response: aiohttp.ClientResponse) -> Optional[int]: + if response.headers.get(self._ratelimit_remaining_header): + return int(response.headers[self._ratelimit_remaining_header]) + + if response.status in self._status_codes_for_ratelimit_hit: + return 0 + + return None + + +class AsyncLimiterMixin(MIXIN_BASE): + """Mixin class that adds rate-limiting behavior to requests.""" + + def __init__( + self, + api_budget: AbstractAPIBudget, + **kwargs: Any, + ): + self._api_budget = api_budget + super().__init__(**kwargs) # type: ignore # Base Session doesn't take any kwargs + + def send(self, request: aiohttp.ClientRequest, **kwargs: Any) -> aiohttp.ClientResponse: + """Send a request with rate-limiting.""" + self._api_budget.acquire_call(request) + response = super().send(request, **kwargs) + self._api_budget.update_from_response(request, response) + return response + + +class AsyncLimiterSession(AsyncLimiterMixin, aiohttp.ClientSession): + """Session that adds rate-limiting behavior to requests.""" + + +class AsyncCachedLimiterSession(aiohttp_client_cache.CachedSession, AsyncLimiterMixin, aiohttp.ClientSession): + """Session class with caching and rate-limiting behavior.""" diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py new file mode 100644 index 000000000000..7fea5a9cedc8 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py @@ -0,0 +1,100 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +import sys +import time +from typing import Any, Callable, Coroutine, Mapping, Optional + +import aiohttp +import backoff + +from .exceptions import DefaultBackoffException, UserDefinedBackoffException + +TRANSIENT_EXCEPTIONS = ( + DefaultBackoffException, + # TODO + # exceptions.ConnectTimeout, + # exceptions.ReadTimeout, + # exceptions.ConnectionError, + # exceptions.ChunkedEncodingError, + aiohttp.ServerTimeoutError, + aiohttp.ServerConnectionError, + aiohttp.ServerDisconnectedError, + +) + +logger = logging.getLogger("airbyte") + + +SendRequestCallableType = Callable[[aiohttp.ClientRequest, Mapping[str, Any]], Coroutine[Any, Any, aiohttp.ClientResponse]] +TOO_MANY_REQUESTS_CODE = 429 + +def default_backoff_handler( + max_tries: Optional[int], factor: float, max_time: Optional[int] = None, **kwargs: Any +) -> Callable[[SendRequestCallableType], SendRequestCallableType]: + def log_retry_attempt(details: Mapping[str, Any]) -> None: + _, exc, _ = sys.exc_info() + if isinstance(exc, aiohttp.ClientResponseError) and exc.history: + logger.info(f"Status code: {exc.status}, Response Content: {'FIX ME'}") # TODO + logger.info( + f"Caught retryable error '{str(exc)}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." + ) + + def should_give_up(exc: Exception) -> bool: + from requests import codes + # If a non-rate-limiting related 4XX error makes it this far, it means it was unexpected and probably consistent, so we shouldn't back off + if isinstance(exc, aiohttp.ClientResponseError): + give_up: bool = ( + exc.history is not None and exc.status != TOO_MANY_REQUESTS_CODE and 400 <= exc.status < 500 + ) + if give_up: + logger.info(f"Giving up for returned HTTP status: {exc.status}") + return give_up + # Only RequestExceptions are retryable, so if we get here, it's not retryable + return False + + return backoff.on_exception( + backoff.expo, + TRANSIENT_EXCEPTIONS, + jitter=None, + on_backoff=log_retry_attempt, + giveup=should_give_up, + max_tries=max_tries, + max_time=max_time, + factor=factor, + **kwargs, + ) + + +def user_defined_backoff_handler( + max_tries: Optional[int], max_time: Optional[int] = None, **kwargs: Any +) -> Callable[[SendRequestCallableType], SendRequestCallableType]: + def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: + _, exc, _ = sys.exc_info() + if isinstance(exc, UserDefinedBackoffException): + if exc.history: + logger.info(f"Status code: {exc.status}, Response Content: {'FIX ME'}") # TODO + retry_after = exc.backoff + logger.info(f"Retrying. Sleeping for {retry_after} seconds") + time.sleep(retry_after + 1) # extra second to cover any fractions of second + + def log_give_up(details: Mapping[str, Any]) -> None: + _, exc, _ = sys.exc_info() + if isinstance(exc, aiohttp.ClientResponseError): + logger.error(f"Max retry limit reached. Request: {exc.request_info}, Response: {exc.history}") # TODO: how does history get printed out + else: + logger.error("Max retry limit reached for unknown request and response") + + return backoff.on_exception( + backoff.constant, + UserDefinedBackoffException, + interval=0, # skip waiting, we'll wait in on_backoff handler + on_backoff=sleep_on_ratelimit, + on_giveup=log_give_up, + jitter=None, + max_tries=max_tries, + max_time=max_time, + **kwargs, + ) From 83a812681284ae49289795a2e2d84bc573c630e9 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:06:03 -0500 Subject: [PATCH 11/74] Working: async adapters for stream --- .../streams/concurrent/adapters_async.py | 429 ++++++++++++++++++ 1 file changed, 429 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py new file mode 100644 index 000000000000..a9210095fdf6 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py @@ -0,0 +1,429 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import copy +import json +import logging +from functools import lru_cache +from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Union + +from airbyte_cdk.models import AirbyteStream, SyncMode +from airbyte_cdk.sources import AbstractSource, Source +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.message import MessageRepository +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy +from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream +from airbyte_cdk.sources.streams.concurrent.availability_strategy import ( + AbstractAvailabilityStrategy, + StreamAvailability, + StreamAvailable, + StreamUnavailable, +) +from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, NoopCursor +from airbyte_cdk.sources.streams.concurrent.default_stream_async import AsyncDefaultStream +from airbyte_cdk.sources.streams.concurrent.exceptions import ExceptionWithDisplayMessage +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator +from airbyte_cdk.sources.streams.concurrent.partitions.record import Record +from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.utils.schema_helpers import InternalConfig +from airbyte_cdk.sources.utils.slice_logger import SliceLogger +from deprecated.classic import deprecated + +""" +This module contains adapters to help enabling concurrency on Stream objects without needing to migrate to AbstractStream +""" + + +@deprecated("This class is experimental. Use at your own risk.") +class AsyncStreamFacade(Stream): + """ + The StreamFacade is a Stream that wraps an AbstractStream and exposes it as a Stream. + + All methods either delegate to the wrapped AbstractStream or provide a default implementation. + The default implementations define restrictions imposed on Streams migrated to the new interface. For instance, only source-defined cursors are supported. + """ + + @classmethod + def create_from_stream( + cls, + stream: Stream, + source: AbstractSource, + logger: logging.Logger, + state: Optional[MutableMapping[str, Any]], + cursor: Cursor, + ) -> Stream: + """ + Create a ConcurrentStream from a Stream object. + :param source: The source + :param stream: The stream + :param max_workers: The maximum number of worker thread to use + :return: + """ + pk = cls._get_primary_key_from_stream(stream.primary_key) + cursor_field = cls._get_cursor_field_from_stream(stream) + + if not source.message_repository: + raise ValueError( + "A message repository is required to emit non-record messages. Please set the message repository on the source." + ) + + message_repository = source.message_repository + return AsyncStreamFacade( + AsyncDefaultStream( + partition_generator=AsyncStreamPartitionGenerator( + stream, + message_repository, + SyncMode.full_refresh if isinstance(cursor, NoopCursor) else SyncMode.incremental, + [cursor_field] if cursor_field is not None else None, + state, + cursor, + ), + name=stream.name, + namespace=stream.namespace, + json_schema=stream.get_json_schema(), + availability_strategy=AsyncStreamAvailabilityStrategy(stream, source), + primary_key=pk, + cursor_field=cursor_field, + logger=logger, + ), + stream, + cursor, + slice_logger=source._slice_logger, + logger=logger, + ) + + @property + def state(self) -> MutableMapping[str, Any]: + raise NotImplementedError("This should not be called as part of the Concurrent CDK code. Please report the problem to Airbyte") + + @state.setter + def state(self, value: Mapping[str, Any]) -> None: + if "state" in dir(self._legacy_stream): + self._legacy_stream.state = value # type: ignore # validating `state` is attribute of stream using `if` above + + @classmethod + def _get_primary_key_from_stream(cls, stream_primary_key: Optional[Union[str, List[str], List[List[str]]]]) -> List[str]: + if stream_primary_key is None: + return [] + elif isinstance(stream_primary_key, str): + return [stream_primary_key] + elif isinstance(stream_primary_key, list): + if len(stream_primary_key) > 0 and all(isinstance(k, str) for k in stream_primary_key): + return stream_primary_key # type: ignore # We verified all items in the list are strings + else: + raise ValueError(f"Nested primary keys are not supported. Found {stream_primary_key}") + else: + raise ValueError(f"Invalid type for primary key: {stream_primary_key}") + + @classmethod + def _get_cursor_field_from_stream(cls, stream: Stream) -> Optional[str]: + if isinstance(stream.cursor_field, list): + if len(stream.cursor_field) > 1: + raise ValueError(f"Nested cursor fields are not supported. Got {stream.cursor_field} for {stream.name}") + elif len(stream.cursor_field) == 0: + return None + else: + return stream.cursor_field[0] + else: + return stream.cursor_field + + def __init__(self, stream: AbstractStream, legacy_stream: Stream, cursor: Cursor, slice_logger: SliceLogger, logger: logging.Logger): + """ + :param stream: The underlying AbstractStream + """ + self._abstract_stream = stream + self._legacy_stream = legacy_stream + self._cursor = cursor + self._slice_logger = slice_logger + self._logger = logger + + async def read_full_refresh( + self, + cursor_field: Optional[List[str]], + logger: logging.Logger, + slice_logger: SliceLogger, + ) -> Iterable[StreamData]: + """ + Read full refresh. Delegate to the underlying AbstractStream, ignoring all the parameters + :param cursor_field: (ignored) + :param logger: (ignored) + :param slice_logger: (ignored) + :return: Iterable of StreamData + """ + async for record in self._read_records(): + yield record + + async def read_incremental( + self, + cursor_field: Optional[List[str]], + logger: logging.Logger, + slice_logger: SliceLogger, + stream_state: MutableMapping[str, Any], + state_manager: ConnectorStateManager, + per_stream_state_enabled: bool, + internal_config: InternalConfig, + ) -> Iterable[StreamData]: + async for record in self._read_records(): + yield record + + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> Iterable[StreamData]: + async for record in self._read_records(): + yield record + + async def _read_records(self) -> Iterable[StreamData]: + async for partition in self._abstract_stream.generate_partitions(): + if self._slice_logger.should_log_slice_message(self._logger): + yield self._slice_logger.create_slice_log_message(partition.to_slice()) + async for record in partition.read(): + yield record.data + + @property + def name(self) -> str: + return self._abstract_stream.name + + @property + def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: + # This method is not expected to be called directly. It is only implemented for backward compatibility with the old interface + return self.as_airbyte_stream().source_defined_primary_key # type: ignore # source_defined_primary_key is known to be an Optional[List[List[str]]] + + @property + def cursor_field(self) -> Union[str, List[str]]: + if self._abstract_stream.cursor_field is None: + return [] + else: + return self._abstract_stream.cursor_field + + @property + def source_defined_cursor(self) -> bool: + # Streams must be aware of their cursor at instantiation time + return True + + @lru_cache(maxsize=None) + def get_json_schema(self) -> Mapping[str, Any]: + return self._abstract_stream.get_json_schema() + + @property + def supports_incremental(self) -> bool: + return self._legacy_stream.supports_incremental + + def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: + """ + Verifies the stream is available. Delegates to the underlying AbstractStream and ignores the parameters + :param logger: (ignored) + :param source: (ignored) + :return: + """ + availability = self._abstract_stream.check_availability() + return availability.is_available(), availability.message() + + def get_error_display_message(self, exception: BaseException) -> Optional[str]: + """ + Retrieves the user-friendly display message that corresponds to an exception. + This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. + + A display message will be returned if the exception is an instance of ExceptionWithDisplayMessage. + + :param exception: The exception that was raised + :return: A user-friendly message that indicates the cause of the error + """ + if isinstance(exception, ExceptionWithDisplayMessage): + return exception.display_message + else: + return None + + def as_airbyte_stream(self) -> AirbyteStream: + return self._abstract_stream.as_airbyte_stream() + + def log_stream_sync_configuration(self) -> None: + self._abstract_stream.log_stream_sync_configuration() + + +class AsyncStreamPartition(Partition): + """ + This class acts as an adapter between the new Partition interface and the Stream's stream_slice interface + + StreamPartitions are instantiated from a Stream and a stream_slice. + + This class can be used to help enable concurrency on existing connectors without having to rewrite everything as AbstractStream. + In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. + """ + + def __init__( + self, + stream: Stream, + _slice: Optional[Mapping[str, Any]], + message_repository: MessageRepository, + sync_mode: SyncMode, + cursor_field: Optional[List[str]], + state: Optional[MutableMapping[str, Any]], + cursor: Cursor, + ): + """ + :param stream: The stream to delegate to + :param _slice: The partition's stream_slice + :param message_repository: The message repository to use to emit non-record messages + """ + self._stream = stream + self._slice = _slice + self._message_repository = message_repository + self._sync_mode = sync_mode + self._cursor_field = cursor_field + self._state = state + self._cursor = cursor + self._is_closed = False + + async def read(self) -> Iterable[Record]: + """ + Read messages from the stream. + If the StreamData is a Mapping, it will be converted to a Record. + Otherwise, the message will be emitted on the message repository. + """ + try: + # using `stream_state=self._state` have a very different behavior than the current one as today the state is updated slice + # by slice incrementally. We don't have this guarantee with Concurrent CDK. For HttpStream, `stream_state` is passed to: + # * fetch_next_page + # * parse_response + # Both are not used for Stripe so we should be good for the first iteration of Concurrent CDK. However, Stripe still do + # `if not stream_state` to know if it calls the Event stream or not + async for record_data in self._stream.read_records( + cursor_field=self._cursor_field, + sync_mode=SyncMode.full_refresh, + stream_slice=copy.deepcopy(self._slice), + stream_state=self._state, + ): + if isinstance(record_data, Mapping): + data_to_return = dict(record_data) + self._stream.transformer.transform(data_to_return, self._stream.get_json_schema()) + record = Record(data_to_return, self._stream.name) + self._cursor.observe(record) + yield Record(data_to_return, self._stream.name) + else: + self._message_repository.emit_message(record_data) + except Exception as e: + display_message = self._stream.get_error_display_message(e) + if display_message: + raise ExceptionWithDisplayMessage(display_message) from e + else: + raise e + + def to_slice(self) -> Optional[Mapping[str, Any]]: + return self._slice + + def __hash__(self) -> int: + if self._slice: + # Convert the slice to a string so that it can be hashed + s = json.dumps(self._slice, sort_keys=True) + return hash((self._stream.name, s)) + else: + return hash(self._stream.name) + + def stream_name(self) -> str: + return self._stream.name + + def close(self) -> None: + self._cursor.close_partition(self) + self._is_closed = True + + def is_closed(self) -> bool: + return self._is_closed + + def __repr__(self) -> str: + return f"StreamPartition({self._stream.name}, {self._slice})" + + +class AsyncStreamPartitionGenerator(PartitionGenerator): + """ + This class acts as an adapter between the new PartitionGenerator and Stream.stream_slices + + This class can be used to help enable concurrency on existing connectors without having to rewrite everything as AbstractStream. + In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. + """ + + def __init__( + self, + stream: Stream, + message_repository: MessageRepository, + sync_mode: SyncMode, + cursor_field: Optional[List[str]], + state: Optional[MutableMapping[str, Any]], + cursor: Cursor, + ): + """ + :param stream: The stream to delegate to + :param message_repository: The message repository to use to emit non-record messages + """ + self.message_repository = message_repository + self._stream = stream + self._sync_mode = sync_mode + self._cursor_field = cursor_field + self._state = state + self._cursor = cursor + + async def generate(self) -> Iterable[Partition]: + for s in self._stream.stream_slices(sync_mode=self._sync_mode, cursor_field=self._cursor_field, stream_state=self._state): + yield AsyncStreamPartition( + self._stream, copy.deepcopy(s), self.message_repository, self._sync_mode, self._cursor_field, self._state, self._cursor + ) + + +@deprecated("This class is experimental. Use at your own risk.") +class AsyncAvailabilityStrategyFacade(AvailabilityStrategy): + def __init__(self, abstract_availability_strategy: AbstractAvailabilityStrategy): + self._abstract_availability_strategy = abstract_availability_strategy + + def check_availability(self, stream: Stream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, Optional[str]]: + """ + Checks stream availability. + + Important to note that the stream and source parameters are not used by the underlying AbstractAvailabilityStrategy. + + :param stream: (unused) + :param logger: logger object to use + :param source: (unused) + :return: A tuple of (boolean, str). If boolean is true, then the stream + """ + stream_availability = self._abstract_availability_strategy.check_availability(logger) + return stream_availability.is_available(), stream_availability.message() + + +class AsyncStreamAvailabilityStrategy(AbstractAvailabilityStrategy): + """ + This class acts as an adapter between the existing AvailabilityStrategy and the new AbstractAvailabilityStrategy. + StreamAvailabilityStrategy is instantiated with a Stream and a Source to allow the existing AvailabilityStrategy to be used with the new AbstractAvailabilityStrategy interface. + + A more convenient implementation would not depend on the docs URL instead of the Source itself, and would support running on an AbstractStream instead of only on a Stream. + + This class can be used to help enable concurrency on existing connectors without having to rewrite everything as AbstractStream and AbstractAvailabilityStrategy. + In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. + """ + + def __init__(self, stream: Stream, source: Source): + """ + :param stream: The stream to delegate to + :param source: The source to delegate to + """ + self._stream = stream + self._source = source + + def check_availability(self, logger: logging.Logger) -> StreamAvailability: + try: + available, message = self._stream.check_availability(logger, self._source) + if available: + return StreamAvailable() + else: + return StreamUnavailable(str(message)) + except Exception as e: + display_message = self._stream.get_error_display_message(e) + if display_message: + raise ExceptionWithDisplayMessage(display_message) + else: + raise e From 48a725d055d992879c8f681c2419b16845cdfc2e Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:08:42 -0500 Subject: [PATCH 12/74] Working: async Salesforce --- .../source_salesforce/source.py | 18 +-- .../source_salesforce/streams.py | 134 ++++++++++++------ 2 files changed, 95 insertions(+), 57 deletions(-) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index 30eea954dfe0..15bca7a60335 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -10,12 +10,11 @@ from airbyte_cdk import AirbyteLogger from airbyte_cdk.logger import AirbyteLogFormatter from airbyte_cdk.models import AirbyteMessage, AirbyteStateMessage, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, Level, SyncMode -from airbyte_cdk.sources.concurrent_source.concurrent_source import ConcurrentSource -from airbyte_cdk.sources.concurrent_source.concurrent_source_adapter import ConcurrentSourceAdapter +from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import InMemoryMessageRepository from airbyte_cdk.sources.streams import Stream -from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade +from airbyte_cdk.sources.streams.concurrent.adapters_async import AsyncStreamFacade from airbyte_cdk.sources.streams.concurrent.cursor import NoopCursor from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator from airbyte_cdk.sources.utils.schema_helpers import InternalConfig @@ -43,7 +42,7 @@ class AirbyteStopSync(AirbyteTracedException): pass -class SourceSalesforce(ConcurrentSourceAdapter): +class SourceSalesforce(AsyncAbstractSource): DATETIME_FORMAT = "%Y-%m-%dT%H:%M:%SZ" START_DATE_OFFSET_IN_YEARS = 2 MAX_WORKERS = 5 @@ -195,11 +194,7 @@ def streams(self, config: Mapping[str, Any]) -> List[Stream]: # TODO: incorporate state & ConcurrentCursor when we support incremental configured_streams = [] for stream in streams: - sync_mode = self._get_sync_mode_from_catalog(stream) - if sync_mode == SyncMode.full_refresh: - configured_streams.append(StreamFacade.create_from_stream(stream, self, logger, None, NoopCursor())) - else: - configured_streams.append(stream) + configured_streams.append(stream) return configured_streams def _get_sync_mode_from_catalog(self, stream: Stream) -> Optional[SyncMode]: @@ -223,7 +218,7 @@ def read( except AirbyteStopSync: logger.info(f"Finished syncing {self.name}") - def _read_stream( + async def _read_stream( self, logger: logging.Logger, stream_instance: Stream, @@ -232,7 +227,8 @@ def _read_stream( internal_config: InternalConfig, ) -> Iterator[AirbyteMessage]: try: - yield from super()._read_stream(logger, stream_instance, configured_stream, state_manager, internal_config) + async for record in super()._read_stream(logger, stream_instance, configured_stream, state_manager, internal_config): + yield record except exceptions.HTTPError as error: error_data = error.response.json()[0] error_code = error_data.get("errorCode") diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 34c03d1caa94..8cabb6db6303 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -2,6 +2,7 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # +import asyncio import csv import ctypes import math @@ -11,15 +12,17 @@ import uuid from abc import ABC from contextlib import closing +from threading import Thread from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Type, Union +import aiohttp import pandas as pd import pendulum import requests # type: ignore[import] from airbyte_cdk.models import ConfiguredAirbyteCatalog, FailureType, SyncMode from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy from airbyte_cdk.sources.streams.core import Stream, StreamData -from airbyte_cdk.sources.streams.http import HttpStream, HttpSubStream +from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream from airbyte_cdk.sources.utils.transform import TransformConfig, TypeTransformer from airbyte_cdk.utils import AirbyteTracedException from numpy import nan @@ -39,7 +42,7 @@ DEFAULT_ENCODING = "utf-8" -class SalesforceStream(HttpStream, ABC): +class SalesforceStream(AsyncHttpStream, ABC): page_size = 2000 transformer = TypeTransformer(TransformConfig.DefaultSchemaNormalization) encoding = DEFAULT_ENCODING @@ -95,8 +98,8 @@ def too_many_properties(self): properties_length = len(urllib.parse.quote(",".join(p for p in selected_properties))) return properties_length > self.max_properties_length - def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: - yield from response.json()["records"] + async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> List[Mapping]: + return (await response.json())["records"] def get_json_schema(self) -> Mapping[str, Any]: if not self.schema: @@ -140,8 +143,8 @@ def path(self, next_page_token: Mapping[str, Any] = None, **kwargs: Any) -> str: return next_token return f"/services/data/{self.sf_api.version}/queryAll" - def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: - response_data = response.json() + async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + response_data = await response.json() next_token = response_data.get("nextRecordsUrl") return {"next_token": next_token} if next_token else None @@ -210,14 +213,30 @@ def _next_chunk_id(property_chunks: Mapping[int, PropertyChunk]) -> Optional[int return None return min(non_exhausted_chunks, key=non_exhausted_chunks.get) - def _read_pages( + async def _read_pages( self, records_generator_fn: Callable[ - [requests.PreparedRequest, requests.Response, Mapping[str, Any], Mapping[str, Any]], Iterable[StreamData] + [aiohttp.ClientRequest, aiohttp.ClientResponse, Mapping[str, Any], Mapping[str, Any]], Iterable[StreamData] ], stream_slice: Mapping[str, Any] = None, stream_state: Mapping[str, Any] = None, - ) -> Iterable[StreamData]: + ) -> None: + self._session = await self._ensure_session() + assert not self._session.closed + # try: + async for record in self._do_read_pages(records_generator_fn, stream_slice, stream_state): + yield record + # finally: + # await self._session.close() + + async def _do_read_pages( + self, + records_generator_fn: Callable[ + [aiohttp.ClientRequest, aiohttp.ClientResponse, Mapping[str, Any], Mapping[str, Any]], Iterable[StreamData] + ], + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ): stream_state = stream_state or {} records_by_primary_key = {} property_chunks: Mapping[int, PropertyChunk] = { @@ -230,15 +249,21 @@ def _read_pages( break property_chunk = property_chunks[chunk_id] - request, response = self._fetch_next_page_for_chunk( - stream_slice, stream_state, property_chunk.next_page, property_chunk.properties - ) + + async def f(): + assert not self._session.closed + request, response = await self._fetch_next_page_for_chunk( + stream_slice, stream_state, property_chunk.next_page, property_chunk.properties + ) + next_page = await self.next_page_token(response) + return request, response, next_page + + request, response, property_chunk.next_page = await f() # When this is the first time we're getting a chunk's records, we set this to False to be used when deciding the next chunk if property_chunk.first_time: property_chunk.first_time = False - property_chunk.next_page = self.next_page_token(response) - chunk_page_records = records_generator_fn(request, response, stream_state, stream_slice) + chunk_page_records = await records_generator_fn(request, response, stream_state, stream_slice) if not self.too_many_properties: # this is the case when a stream has no primary key # (it is allowed when properties length does not exceed the maximum value) @@ -276,16 +301,14 @@ def _read_pages( if incomplete_record_ids: self.logger.warning(f"Inconsistent record(s) with primary keys {incomplete_record_ids} found. Skipping them.") - # Always return an empty generator just in case no records were ever yielded - yield from [] - - def _fetch_next_page_for_chunk( + async def _fetch_next_page_for_chunk( self, stream_slice: Mapping[str, Any] = None, stream_state: Mapping[str, Any] = None, next_page_token: Mapping[str, Any] = None, property_chunk: Mapping[str, Any] = None, - ) -> Tuple[requests.PreparedRequest, requests.Response]: + ) -> Tuple[aiohttp.ClientRequest, aiohttp.ClientResponse]: + assert not self._session.closed request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) request = self._create_prepared_request( path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), @@ -297,7 +320,7 @@ def _fetch_next_page_for_chunk( data=self.request_body_data(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), ) request_kwargs = self.request_kwargs(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) - response = self._send_request(request, request_kwargs) + response = await self._send_request(request, request_kwargs) return request, response @@ -336,24 +359,24 @@ def path(self, next_page_token: Mapping[str, Any] = None, **kwargs: Any) -> str: transformer = TypeTransformer(TransformConfig.CustomSchemaNormalization | TransformConfig.DefaultSchemaNormalization) @default_backoff_handler(max_tries=5, factor=15) - def _send_http_request(self, method: str, url: str, json: dict = None, headers: dict = None, stream: bool = False): + async def _send_http_request(self, method: str, url: str, json: dict = None, headers: dict = None, stream=False) -> aiohttp.ClientResponse: headers = self.authenticator.get_auth_header() if not headers else headers | self.authenticator.get_auth_header() - response = self._session.request(method, url=url, headers=headers, json=json, stream=stream) - if response.status_code not in [200, 204]: - self.logger.error(f"error body: {response.text}, sobject options: {self.sobject_options}") + response = await self._session.request(method, url=url, headers=headers, json=json) + if response.status not in [200, 204]: + self.logger.error(f"error body: {await response.text()}, sobject options: {self.sobject_options}") response.raise_for_status() - return response + return response # TODO: how to handle the stream argument - def create_stream_job(self, query: str, url: str) -> Optional[str]: + async def create_stream_job(self, query: str, url: str) -> Optional[str]: """ docs: https://developer.salesforce.com/docs/atlas.en-us.api_asynch.meta/api_asynch/create_job.html """ json = {"operation": "queryAll", "query": query, "contentType": "CSV", "columnDelimiter": "COMMA", "lineEnding": "LF"} try: - response = self._send_http_request("POST", url, json=json) - job_id: str = response.json()["id"] + response = await self._send_http_request("POST", url, json=json) + job_id: str = (await response.json())["id"] return job_id - except exceptions.HTTPError as error: + except exceptions.HTTPError as error: # TODO: which errors? if error.response.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: # A part of streams can't be used by BULK API. Every API version can have a custom list of # these sobjects. Another part of them can be generated dynamically. That's why we can't track @@ -408,7 +431,7 @@ def create_stream_job(self, query: str, url: str) -> Optional[str]: raise error return None - def wait_for_job(self, url: str) -> str: + async def wait_for_job(self, url: str) -> str: expiration_time: DateTime = pendulum.now().add(seconds=self.DEFAULT_WAIT_TIMEOUT_SECONDS) job_status = "InProgress" delay_timeout = 0.0 @@ -419,7 +442,7 @@ def wait_for_job(self, url: str) -> str: time.sleep(0.5) while pendulum.now() < expiration_time: try: - job_info = self._send_http_request("GET", url=url).json() + job_info = await (await self._send_http_request("GET", url=url)).json() except exceptions.HTTPError as error: error_data = error.response.json()[0] error_code = error_data.get("errorCode") @@ -457,14 +480,14 @@ def wait_for_job(self, url: str) -> str: self.logger.warning(f"Not wait the {self.name} data for {self.DEFAULT_WAIT_TIMEOUT_SECONDS} seconds, data: {job_info}!!") return job_status - def execute_job(self, query: str, url: str) -> Tuple[Optional[str], Optional[str]]: + async def execute_job(self, query: str, url: str) -> Tuple[Optional[str], Optional[str]]: job_status = "Failed" for i in range(0, self.MAX_RETRY_NUMBER): - job_id = self.create_stream_job(query=query, url=url) + job_id = await self.create_stream_job(query=query, url=url) if not job_id: return None, job_status job_full_url = f"{url}/{job_id}" - job_status = self.wait_for_job(url=job_full_url) + job_status = await self.wait_for_job(url=job_full_url) if job_status not in ["UploadComplete", "InProgress"]: break self.logger.error(f"Waiting error. Try to run this job again {i + 1}/{self.MAX_RETRY_NUMBER}...") @@ -472,7 +495,7 @@ def execute_job(self, query: str, url: str) -> Tuple[Optional[str], Optional[str job_status = "Aborted" if job_status in ["Aborted", "Failed"]: - self.delete_job(url=job_full_url) + await self.delete_job(url=job_full_url) return None, job_status return job_full_url, job_status @@ -504,7 +527,7 @@ def get_response_encoding(self, headers) -> str: return self.encoding - def download_data(self, url: str, chunk_size: int = 1024) -> tuple[str, str, dict]: + async def download_data(self, url: str, chunk_size: int = 1024) -> tuple[str, str, dict]: """ Retrieves binary data result from successfully `executed_job`, using chunks, to avoid local memory limitations. @ url: string - the url of the `executed_job` @@ -513,12 +536,13 @@ def download_data(self, url: str, chunk_size: int = 1024) -> tuple[str, str, dic """ # set filepath for binary data from response tmp_file = str(uuid.uuid4()) - with closing(self._send_http_request("GET", url, headers={"Accept-Encoding": "gzip"}, stream=True)) as response, open( + response = await self._send_http_request("GET", url, headers={"Accept-Encoding": "gzip"}, stream=True) + with open( tmp_file, "wb" ) as data_file: response_headers = response.headers response_encoding = self.get_response_encoding(response_headers) - for chunk in response.iter_content(chunk_size=chunk_size): + async for chunk in response.content.iter_chunked(chunk_size): data_file.write(self.filter_null_bytes(chunk)) # check the file exists if os.path.isfile(tmp_file): @@ -554,8 +578,8 @@ def abort_job(self, url: str): self._send_http_request("PATCH", url=url, json=data) self.logger.warning("Broken job was aborted") - def delete_job(self, url: str): - self._send_http_request("DELETE", url=url) + async def delete_job(self, url: str): + await self._send_http_request("DELETE", url=url) @property def availability_strategy(self) -> Optional["AvailabilityStrategy"]: @@ -593,19 +617,34 @@ def request_params( return {"q": query} - def read_records( + async def read_records( self, sync_mode: SyncMode, cursor_field: List[str] = None, stream_slice: Mapping[str, Any] = None, stream_state: Mapping[str, Any] = None, ) -> Iterable[Mapping[str, Any]]: + self._session = await self._ensure_session() + assert not self._session.closed + # try: + async for record in self._do_read_records(sync_mode, cursor_field, stream_slice, stream_state): + yield record + # finally: + # await self._session.close() + + async def _do_read_records( + self, + sync_mode: SyncMode, + cursor_field: List[str] = None, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ): stream_state = stream_state or {} next_page_token = None params = self.request_params(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) path = self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) - job_full_url, job_status = self.execute_job(query=params["q"], url=f"{self.url_base}{path}") + job_full_url, job_status = await self.execute_job(query=params["q"], url=f"{self.url_base}{path}") if not job_full_url: if job_status == "Failed": # As rule as BULK logic returns unhandled error. For instance: @@ -618,23 +657,25 @@ def read_records( if not stream_is_available: self.logger.warning(f"Skipped syncing stream '{standard_instance.name}' because it was unavailable. Error: {error}") return - yield from standard_instance.read_records( + for record in standard_instance.read_records( sync_mode=sync_mode, cursor_field=cursor_field, stream_slice=stream_slice, stream_state=stream_state - ) + ): + yield record return raise SalesforceException(f"Job for {self.name} stream using BULK API was failed.") salesforce_bulk_api_locator = None while True: req = PreparedRequest() req.prepare_url(f"{job_full_url}/results", {"locator": salesforce_bulk_api_locator}) - tmp_file, response_encoding, response_headers = self.download_data(url=req.url) + tmp_file, response_encoding, response_headers = await self.download_data(url=req.url) for record in self.read_with_chunks(tmp_file, response_encoding): yield record if response_headers.get("Sforce-Locator", "null") == "null": break salesforce_bulk_api_locator = response_headers.get("Sforce-Locator") - self.delete_job(url=job_full_url) + + await self.delete_job(url=job_full_url) def get_standard_instance(self) -> SalesforceStream: """Returns a instance of standard logic(non-BULK) with same settings""" @@ -645,6 +686,7 @@ def get_standard_instance(self) -> SalesforceStream: schema=self.schema, sobject_options=self.sobject_options, authenticator=self.authenticator, + session=self._session, ) new_cls: Type[SalesforceStream] = RestSalesforceStream if isinstance(self, BulkIncrementalSalesforceStream): From 52ca10227489f11ed54c0809f19f9ef1406c5331 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:09:07 -0500 Subject: [PATCH 13/74] Code for local testing --- airbyte-cdk/python/airbyte_cdk/entrypoint.py | 2 +- .../integration_tests/configured_catalog.json | 121 +----------------- .../source_salesforce/api.py | 2 +- 3 files changed, 3 insertions(+), 122 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/entrypoint.py b/airbyte-cdk/python/airbyte_cdk/entrypoint.py index f89e0ef0ec29..6ee464eef57d 100644 --- a/airbyte-cdk/python/airbyte_cdk/entrypoint.py +++ b/airbyte-cdk/python/airbyte_cdk/entrypoint.py @@ -211,7 +211,7 @@ def launch(source: Source, args: List[str]) -> None: source_entrypoint = AirbyteEntrypoint(source) parsed_args = source_entrypoint.parse_args(args) for message in source_entrypoint.run(parsed_args): - print(message) + print(message[:50]) def _init_internal_request_filter() -> None: diff --git a/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json b/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json index 1d82d43b11a1..53f4ded6de8f 100644 --- a/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json +++ b/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json @@ -1,120 +1 @@ -{ - "streams": [ - { - "stream": { - "name": "Account", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "ActiveFeatureLicenseMetric", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "ActivePermSetLicenseMetric", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "ActiveProfileMetric", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "AppDefinition", - "json_schema": {}, - "supported_sync_modes": ["full_refresh"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "overwrite" - }, - { - "stream": { - "name": "Asset", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "FormulaFunctionAllowedType", - "json_schema": {}, - "supported_sync_modes": ["full_refresh"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "overwrite" - }, - { - "stream": { - "name": "ObjectPermissions", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "PermissionSetTabSetting", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["SystemModstamp"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - }, - { - "stream": { - "name": "LeadHistory", - "json_schema": {}, - "supported_sync_modes": ["full_refresh", "incremental"], - "source_defined_cursor": true, - "default_cursor_field": ["CreatedDate"], - "source_defined_primary_key": [["Id"]] - }, - "sync_mode": "full_refresh", - "destination_sync_mode": "append" - } - ] -} +{"streams": [{"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}]} \ No newline at end of file diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py index eb0eed9ef70d..92bd614c7a2d 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py @@ -283,7 +283,7 @@ def get_validated_streams(self, config: Mapping[str, Any], catalog: ConfiguredAi if catalog: return { - configured_stream.stream.name: stream_objects[configured_stream.stream.name] + configured_stream.stream.name: stream_objects["Account"] for configured_stream in catalog.streams if configured_stream.stream.name in stream_objects } From eef1b4f41aa870af3d45f27ae78e61344ec03b3c Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:15:53 -0500 Subject: [PATCH 14/74] Working but not really used: AsyncPartition --- .../sources/streams/concurrent/adapters.py | 10 +-- .../concurrent/partitions/partition_async.py | 63 +++++++++++++++++++ 2 files changed, 68 insertions(+), 5 deletions(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py index f8a5e3ed65e3..ec593f1bc0d5 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py @@ -24,8 +24,8 @@ from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, NoopCursor from airbyte_cdk.sources.streams.concurrent.default_stream import DefaultStream from airbyte_cdk.sources.streams.concurrent.exceptions import ExceptionWithDisplayMessage -from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition -from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator +from airbyte_cdk.sources.streams.concurrent.partitions.partition_async import AsyncPartition +from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator_async import AsyncPartitionGenerator from airbyte_cdk.sources.streams.concurrent.partitions.record import Record from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.utils.schema_helpers import InternalConfig @@ -255,7 +255,7 @@ def log_stream_sync_configuration(self) -> None: self._abstract_stream.log_stream_sync_configuration() -class StreamPartition(Partition): +class StreamPartition(AsyncPartition): """ This class acts as an adapter between the new Partition interface and the Stream's stream_slice interface @@ -348,7 +348,7 @@ def __repr__(self) -> str: return f"StreamPartition({self._stream.name}, {self._slice})" -class StreamPartitionGenerator(PartitionGenerator): +class StreamPartitionGenerator(AsyncPartitionGenerator): """ This class acts as an adapter between the new PartitionGenerator and Stream.stream_slices @@ -376,7 +376,7 @@ def __init__( self._state = state self._cursor = cursor - def generate(self) -> Iterable[Partition]: + def generate(self) -> Iterable[AsyncPartition]: for s in self._stream.stream_slices(sync_mode=self._sync_mode, cursor_field=self._cursor_field, stream_state=self._state): yield StreamPartition( self._stream, copy.deepcopy(s), self.message_repository, self._sync_mode, self._cursor_field, self._state, self._cursor diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py new file mode 100644 index 000000000000..df7f77dd2411 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py @@ -0,0 +1,63 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +from abc import ABC, abstractmethod +from typing import Any, Iterable, Mapping, Optional + +from airbyte_cdk.sources.streams.concurrent.partitions.record import Record + + +class AsyncPartition(ABC): + """ + A partition is responsible for reading a specific set of data from a source. + """ + + @abstractmethod + async def read(self) -> Iterable[Record]: + """ + Reads the data from the partition. + :return: An iterable of records. + """ + pass + + @abstractmethod + def to_slice(self) -> Optional[Mapping[str, Any]]: + """ + Converts the partition to a slice that can be serialized and deserialized. + + Note: it would have been interesting to have a type of `Mapping[str, Comparable]` to simplify typing but some slices can have nested + values ([example](https://github.com/airbytehq/airbyte/blob/1ce84d6396e446e1ac2377362446e3fb94509461/airbyte-integrations/connectors/source-stripe/source_stripe/streams.py#L584-L596)) + :return: A mapping representing a slice + """ + pass + + @abstractmethod + def stream_name(self) -> str: + """ + Returns the name of the stream that this partition is reading from. + :return: The name of the stream. + """ + pass + + @abstractmethod + def close(self) -> None: + """ + Closes the partition. + """ + pass + + @abstractmethod + def is_closed(self) -> bool: + """ + Returns whether the partition is closed. + :return: + """ + pass + + @abstractmethod + def __hash__(self) -> int: + """ + Returns a hash of the partition. + Partitions must be hashable so that they can be used as keys in a dictionary. + """ From a6493ea3718bb68e627e34be5ed998a2554cf3d3 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 21:32:38 -0500 Subject: [PATCH 15/74] Use AsyncStream instead of Stream --- .../sources/streams/async_call_rate.py | 1 - .../streams/concurrent/adapters_async.py | 28 +++++++++--------- .../airbyte_cdk/sources/streams/core_async.py | 29 ++++++++++++++++++- 3 files changed, 42 insertions(+), 16 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py index edb024724736..24b1a81fcd47 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py @@ -3,7 +3,6 @@ # import abc -import dataclasses import datetime import logging import time diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py index a9210095fdf6..5e042be63ca8 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py @@ -12,7 +12,7 @@ from airbyte_cdk.sources import AbstractSource, Source from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import MessageRepository -from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.core_async import AsyncStream from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream from airbyte_cdk.sources.streams.concurrent.availability_strategy import ( @@ -38,7 +38,7 @@ @deprecated("This class is experimental. Use at your own risk.") -class AsyncStreamFacade(Stream): +class AsyncStreamFacade(AsyncStream): """ The StreamFacade is a Stream that wraps an AbstractStream and exposes it as a Stream. @@ -49,12 +49,12 @@ class AsyncStreamFacade(Stream): @classmethod def create_from_stream( cls, - stream: Stream, + stream: AsyncStream, source: AbstractSource, logger: logging.Logger, state: Optional[MutableMapping[str, Any]], cursor: Cursor, - ) -> Stream: + ) -> AsyncStream: """ Create a ConcurrentStream from a Stream object. :param source: The source @@ -119,7 +119,7 @@ def _get_primary_key_from_stream(cls, stream_primary_key: Optional[Union[str, Li raise ValueError(f"Invalid type for primary key: {stream_primary_key}") @classmethod - def _get_cursor_field_from_stream(cls, stream: Stream) -> Optional[str]: + def _get_cursor_field_from_stream(cls, stream: AsyncStream) -> Optional[str]: if isinstance(stream.cursor_field, list): if len(stream.cursor_field) > 1: raise ValueError(f"Nested cursor fields are not supported. Got {stream.cursor_field} for {stream.name}") @@ -130,7 +130,7 @@ def _get_cursor_field_from_stream(cls, stream: Stream) -> Optional[str]: else: return stream.cursor_field - def __init__(self, stream: AbstractStream, legacy_stream: Stream, cursor: Cursor, slice_logger: SliceLogger, logger: logging.Logger): + def __init__(self, stream: AbstractStream, legacy_stream: AsyncStream, cursor: Cursor, slice_logger: SliceLogger, logger: logging.Logger): """ :param stream: The underlying AbstractStream """ @@ -259,7 +259,7 @@ class AsyncStreamPartition(Partition): def __init__( self, - stream: Stream, + stream: AsyncStream, _slice: Optional[Mapping[str, Any]], message_repository: MessageRepository, sync_mode: SyncMode, @@ -309,7 +309,7 @@ async def read(self) -> Iterable[Record]: else: self._message_repository.emit_message(record_data) except Exception as e: - display_message = self._stream.get_error_display_message(e) + display_message = await self._stream.get_error_display_message(e) if display_message: raise ExceptionWithDisplayMessage(display_message) from e else: @@ -350,7 +350,7 @@ class AsyncStreamPartitionGenerator(PartitionGenerator): def __init__( self, - stream: Stream, + stream: AsyncStream, message_repository: MessageRepository, sync_mode: SyncMode, cursor_field: Optional[List[str]], @@ -380,7 +380,7 @@ class AsyncAvailabilityStrategyFacade(AvailabilityStrategy): def __init__(self, abstract_availability_strategy: AbstractAvailabilityStrategy): self._abstract_availability_strategy = abstract_availability_strategy - def check_availability(self, stream: Stream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, Optional[str]]: + def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, Optional[str]]: """ Checks stream availability. @@ -406,7 +406,7 @@ class AsyncStreamAvailabilityStrategy(AbstractAvailabilityStrategy): In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. """ - def __init__(self, stream: Stream, source: Source): + def __init__(self, stream: AsyncStream, source: Source): """ :param stream: The stream to delegate to :param source: The source to delegate to @@ -414,15 +414,15 @@ def __init__(self, stream: Stream, source: Source): self._stream = stream self._source = source - def check_availability(self, logger: logging.Logger) -> StreamAvailability: + async def check_availability(self, logger: logging.Logger) -> StreamAvailability: try: - available, message = self._stream.check_availability(logger, self._source) + available, message = await self._stream.check_availability(logger, self._source) if available: return StreamAvailable() else: return StreamUnavailable(str(message)) except Exception as e: - display_message = self._stream.get_error_display_message(e) + display_message = await self._stream.get_error_display_message(e) if display_message: raise ExceptionWithDisplayMessage(display_message) else: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py index 881e2809383a..ca8be5fea116 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -5,7 +5,7 @@ import logging from abc import ABC, abstractmethod -from typing import Any, Iterable, List, Mapping, MutableMapping, Optional +from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Tuple from airbyte_cdk.models import SyncMode from airbyte_cdk.models import Type as MessageType @@ -112,3 +112,30 @@ async def stream_slices( :return: """ return [None] + + async def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: + """ + Checks whether this stream is available. + + :param logger: source logger + :param source: (optional) source + :return: A tuple of (boolean, str). If boolean is true, then this stream + is available, and no str is required. Otherwise, this stream is unavailable + for some reason and the str should describe what went wrong and how to + resolve the unavailability, if possible. + """ + if self.availability_strategy: + return self.availability_strategy.check_availability(self, logger, source) + return True, None + + async def get_error_display_message(self, exception: BaseException) -> Optional[str]: + """ + Retrieves the user-friendly display message that corresponds to an exception. + This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. + + The default implementation of this method does not return user-friendly messages for any exception type, but it should be overriden as needed. + + :param exception: The exception that was raised + :return: A user-friendly message that indicates the cause of the error + """ + return None From 37b049c3174ab8b61b37d998611446ec001d936e Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 22:16:37 -0500 Subject: [PATCH 16/74] Working: use Limiter sessions (but need to test limiters) --- .../sources/streams/async_call_rate.py | 4 +- .../sources/streams/http/http_async.py | 39 +++++++------------ 2 files changed, 16 insertions(+), 27 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py index 24b1a81fcd47..40e3a32f43c7 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py @@ -483,10 +483,10 @@ def __init__( self._api_budget = api_budget super().__init__(**kwargs) # type: ignore # Base Session doesn't take any kwargs - def send(self, request: aiohttp.ClientRequest, **kwargs: Any) -> aiohttp.ClientResponse: + async def send(self, request: aiohttp.ClientRequest, **kwargs: Any) -> aiohttp.ClientResponse: """Send a request with rate-limiting.""" self._api_budget.acquire_call(request) - response = super().send(request, **kwargs) + response = await super().send(request, **kwargs) self._api_budget.update_from_response(request, response) return response diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 1e8b04d7d96f..43c3f91c6792 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -43,6 +43,7 @@ class AsyncHttpStream(HttpStream, AsyncStream, ABC): def __init__(self, authenticator: Optional[Union[BasicAuth, HttpAuthenticator]] = None, api_budget: Optional[APIBudget] = None): self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) self._session: aiohttp.ClientSession = None + # self._session: aiohttp.ClientSession = self.request_session() assert authenticator self._authenticator = authenticator # TODO: handle the preexisting code paths @@ -51,7 +52,17 @@ def request_session(self) -> aiohttp.ClientSession: Session factory based on use_cache property and call rate limits (api_budget parameter) :return: instance of request-based session """ + connector = aiohttp.TCPConnector( + limit_per_host=MAX_CONNECTION_POOL_SIZE, + limit=MAX_CONNECTION_POOL_SIZE, + ) + kwargs = {} + assert self._authenticator.get_auth_header() + if self._authenticator: + kwargs['headers'] = self._authenticator.get_auth_header() + if self.use_cache: + raise NotImplementedError("TODO: test this codepath") cache_dir = os.getenv(ENV_REQUEST_CACHE_PATH) # Use in-memory cache if cache_dir is not set # This is a non-obvious interface, but it ensures we don't write sql files when running unit tests @@ -59,9 +70,9 @@ def request_session(self) -> aiohttp.ClientSession: sqlite_path = str(Path(cache_dir) / self.cache_filename) else: sqlite_path = "file::memory:?cache=shared" - return AsyncCachedLimiterSession(sqlite_path, backend="sqlite", api_budget=self._api_budget) # type: ignore # there are no typeshed stubs for requests_cache + return AsyncCachedLimiterSession(sqlite_path, backend="sqlite", connector=connector, api_budget=self._api_budget) # type: ignore # there are no typeshed stubs for requests_cache else: - return AsyncLimiterSession(api_budget=self._api_budget) + return AsyncLimiterSession(connector=connector, api_budget=self._api_budget, **kwargs) def clear_cache(self) -> None: """ @@ -220,30 +231,8 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st return response async def _ensure_session(self) -> aiohttp.ClientSession: - - # if self.use_cache: - # cache_dir = os.getenv(ENV_REQUEST_CACHE_PATH) - # # Use in-memory cache if cache_dir is not set - # # This is a non-obvious interface, but it ensures we don't write sql files when running unit tests - # if cache_dir: - # sqlite_path = str(Path(cache_dir) / self.cache_filename) - # else: - # sqlite_path = "file::memory:?cache=shared" - # self._session = AsyncCachedLimiterSession(sqlite_path, backend="sqlite", api_budget=self._api_budget) # type: ignore # there are no typeshed stubs for requests_cache - # else: - # self._session = AsyncLimiterSession(api_budget=self._api_budget) - if self._session is None or self._session.closed: # TODO: why is the session closing? - # TODO: figure out caching of session & requests - connector = aiohttp.TCPConnector( - limit_per_host=MAX_CONNECTION_POOL_SIZE, - limit=MAX_CONNECTION_POOL_SIZE, - ) - kwargs = {} - assert self._authenticator.get_auth_header() - if self._authenticator: - kwargs['headers'] = self._authenticator.get_auth_header() - self._session = aiohttp.ClientSession(connector=connector, **kwargs) + self._session = self.request_session() return self._session async def _send_request(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any]) -> aiohttp.ClientResponse: From d928fff7ff2a02f48fcf08f99f7dc1f6e75a7daf Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 22:33:36 -0500 Subject: [PATCH 17/74] Add json header if sending json data --- .../airbyte_cdk/sources/streams/http/http_async.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 43c3f91c6792..7285531899f4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -162,6 +162,7 @@ def _create_aiohttp_client_request( json_data: Optional[Mapping[str, Any]] = None, data: Optional[Union[str, Mapping[str, Any]]] = None, ) -> aiohttp.ClientRequest: + str_url = self._join_url(self.url_base, path) str_url = "http://localhost:8000" url = URL(str_url) if self.must_deduplicate_query_params(): @@ -173,10 +174,12 @@ def _create_aiohttp_client_request( raise RequestBodyException( "At the same time only one of the 'request_body_data' and 'request_body_json' functions can return data" ) + elif json_data: + headers = headers or {} + headers.update({'Content-Type': 'application/json'}) + data = json.dumps(json_data) - client_request = aiohttp.ClientRequest( - self.http_method, url, headers=headers, params=query_params, data=json.dumps(json_data) if json_data else data - ) # TODO: add json header if json_data? + client_request = aiohttp.ClientRequest(self.http_method, url, headers=headers, params=query_params, data=data) return client_request @@ -280,7 +283,7 @@ async def send(): return await send() @classmethod - async def parse_response_error_message(cls, response: aiohttp.ClientResponse) -> Optional[str]: # TODO + async def parse_response_error_message(cls, response: aiohttp.ClientResponse) -> Optional[str]: """ Parses the raw response object from a failed request into a user-friendly error message. By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. @@ -288,7 +291,6 @@ async def parse_response_error_message(cls, response: aiohttp.ClientResponse) -> :param response: :return: A user-friendly message that indicates the cause of the error """ - # default logic to grab error from common fields def _try_get_error(value: Optional[JsonType]) -> Optional[str]: if isinstance(value, str): From aaf19cfd9f79ceaafaf72d791cde6d0765f797ed Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 22:47:43 -0500 Subject: [PATCH 18/74] Update request args --- .../airbyte_cdk/sources/streams/http/http_async.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 7285531899f4..a86287bbd313 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -206,8 +206,14 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st "Making outbound API request", extra={"headers": request.headers, "url": request.url, "request_body": request.body} ) - # TODO: get headers and anything else off of request & combine with request_kwargs? - response = await self._session.request(request.method, request.url, **request_kwargs) + response = await self._session.request( + request.method, request.url, + headers=request.headers, + auth=request.auth, + chunked=request.chunked, + compress=request.compress, + **request_kwargs, + ) # Evaluation of response.text can be heavy, for example, if streaming a large response # Do it only in debug mode From 5f165811ebd09040cae332bd790da326067526bb Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 22 Dec 2023 23:05:24 -0500 Subject: [PATCH 19/74] Fix aiohttp cache --- .../python/airbyte_cdk/sources/streams/http/http_async.py | 6 +++--- airbyte-cdk/python/setup.py | 2 ++ 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index a86287bbd313..bc10868e965a 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -62,7 +62,6 @@ def request_session(self) -> aiohttp.ClientSession: kwargs['headers'] = self._authenticator.get_auth_header() if self.use_cache: - raise NotImplementedError("TODO: test this codepath") cache_dir = os.getenv(ENV_REQUEST_CACHE_PATH) # Use in-memory cache if cache_dir is not set # This is a non-obvious interface, but it ensures we don't write sql files when running unit tests @@ -70,7 +69,8 @@ def request_session(self) -> aiohttp.ClientSession: sqlite_path = str(Path(cache_dir) / self.cache_filename) else: sqlite_path = "file::memory:?cache=shared" - return AsyncCachedLimiterSession(sqlite_path, backend="sqlite", connector=connector, api_budget=self._api_budget) # type: ignore # there are no typeshed stubs for requests_cache + cache = aiohttp_client_cache.SQLiteBackend(cache_dir=sqlite_path) + return AsyncCachedLimiterSession(cache=cache, connector=connector, api_budget=self._api_budget) else: return AsyncLimiterSession(connector=connector, api_budget=self._api_budget, **kwargs) @@ -390,7 +390,7 @@ async def _fetch_next_page( stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, next_page_token: Optional[Mapping[str, Any]] = None, - ) -> Tuple[aiohttp.ClientRequest, aiohttp.ClientResponse]: # TODO: maybe don't need to return request too since its on aiohttp.ClientResponse + ) -> Tuple[aiohttp.ClientRequest, aiohttp.ClientResponse]: request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) request = self._create_prepared_request( path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), diff --git a/airbyte-cdk/python/setup.py b/airbyte-cdk/python/setup.py index a5cac26e35fd..61aab4a298cd 100644 --- a/airbyte-cdk/python/setup.py +++ b/airbyte-cdk/python/setup.py @@ -65,6 +65,8 @@ packages=find_packages(exclude=("unit_tests",)), package_data={"airbyte_cdk": ["py.typed", "sources/declarative/declarative_component_schema.yaml"]}, install_requires=[ + "aiohttp-client-cache[aiosqlite]", + "aiosqlite", "airbyte-protocol-models==0.5.1", "backoff", "dpath~=2.0.1", From 343ba2bed5097d1631bd5b4530939e9f88749dd2 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 24 Dec 2023 11:07:35 -0500 Subject: [PATCH 20/74] Working: handle token-based auth --- .../sources/streams/http/http_async.py | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index bc10868e965a..838c05729dbb 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -12,7 +12,6 @@ import aiohttp import aiohttp_client_cache -from aiohttp import BasicAuth from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession @@ -24,7 +23,7 @@ from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH -from .auth.core import HttpAuthenticator, NoAuth +from .auth.core import HttpAuthenticator from .rate_limiting import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies @@ -39,13 +38,16 @@ class AsyncHttpStream(HttpStream, AsyncStream, ABC): Basic building block for users building an Airbyte source for an async HTTP API. """ - # TODO: remove legacy HttpAuthenticator authenticator references - def __init__(self, authenticator: Optional[Union[BasicAuth, HttpAuthenticator]] = None, api_budget: Optional[APIBudget] = None): + def __init__(self, authenticator: HttpAuthenticator, api_budget: Optional[APIBudget] = None): self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) self._session: aiohttp.ClientSession = None # self._session: aiohttp.ClientSession = self.request_session() - assert authenticator - self._authenticator = authenticator # TODO: handle the preexisting code paths + # TODO: HttpStream handles other authentication codepaths, which may need to be added later + self._authenticator = authenticator + + @property + def authenticator(self) -> HttpAuthenticator: + return self._authenticator def request_session(self) -> aiohttp.ClientSession: """ @@ -57,7 +59,7 @@ def request_session(self) -> aiohttp.ClientSession: limit=MAX_CONNECTION_POOL_SIZE, ) kwargs = {} - assert self._authenticator.get_auth_header() + if self._authenticator: kwargs['headers'] = self._authenticator.get_auth_header() From b8675a77d5b300310928b03ff8ed171c7e9dc4c2 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 24 Dec 2023 11:08:27 -0500 Subject: [PATCH 21/74] Working: refactor BaseHttpStream out of HttpStream & AsyncHttpStream --- .../airbyte_cdk/sources/streams/http/http.py | 154 +-------- .../sources/streams/http/http_async.py | 4 +- .../sources/streams/http/http_base.py | 305 ++++++++++++++++++ 3 files changed, 309 insertions(+), 154 deletions(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py index e5784cd25c03..e588d679035d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py @@ -19,6 +19,7 @@ from airbyte_cdk.sources.streams.call_rate import APIBudget, CachedLimiterSession, LimiterSession from airbyte_cdk.sources.streams.core import Stream, StreamData from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy +from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH from requests.auth import AuthBase @@ -31,7 +32,7 @@ BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") -class HttpStream(Stream, ABC): +class HttpStream(BaseHttpStream, ABC): """ Base abstract class for an Airbyte Stream using the HTTP protocol. Basic building block for users building an Airbyte source for a HTTP API. """ @@ -52,22 +53,6 @@ def __init__(self, authenticator: Optional[Union[AuthBase, HttpAuthenticator]] = elif authenticator: self._authenticator = authenticator - @property - def cache_filename(self) -> str: - """ - Override if needed. Return the name of cache file - Note that if the environment variable REQUEST_CACHE_PATH is not set, the cache will be in-memory only. - """ - return f"{self.name}.sqlite" - - @property - def use_cache(self) -> bool: - """ - Override if needed. If True, all records will be cached. - Note that if the environment variable REQUEST_CACHE_PATH is not set, the cache will be in-memory only. - """ - return False - def request_session(self) -> requests.Session: """ Session factory based on use_cache property and call rate limits (api_budget parameter) @@ -92,56 +77,10 @@ def clear_cache(self) -> None: if isinstance(self._session, requests_cache.CachedSession): self._session.cache.clear() # type: ignore # cache.clear is not typed - @property - @abstractmethod - def url_base(self) -> str: - """ - :return: URL base for the API endpoint e.g: if you wanted to hit https://myapi.com/v1/some_entity then this should return "https://myapi.com/v1/" - """ - - @property - def http_method(self) -> str: - """ - Override if needed. See get_request_data/get_request_json if using POST/PUT/PATCH. - """ - return "GET" - - @property - def raise_on_http_errors(self) -> bool: - """ - Override if needed. If set to False, allows opting-out of raising HTTP code exception. - """ - return True - - @property - def max_retries(self) -> Union[int, None]: - """ - Override if needed. Specifies maximum amount of retries for backoff policy. Return None for no limit. - """ - return 5 - - @property - def max_time(self) -> Union[int, None]: - """ - Override if needed. Specifies maximum total waiting time (in seconds) for backoff policy. Return None for no limit. - """ - return 60 * 10 - - @property - def retry_factor(self) -> float: - """ - Override if needed. Specifies factor for backoff policy. - """ - return 5 - @property def authenticator(self) -> HttpAuthenticator: return self._authenticator - @property - def availability_strategy(self) -> Optional[AvailabilityStrategy]: - return HttpAvailabilityStrategy() - @abstractmethod def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: """ @@ -164,73 +103,6 @@ def path( Returns the URL path for the API endpoint e.g: if you wanted to hit https://myapi.com/v1/some_entity then this should return "some_entity" """ - def request_params( - self, - stream_state: Optional[Mapping[str, Any]], - stream_slice: Optional[Mapping[str, Any]] = None, - next_page_token: Optional[Mapping[str, Any]] = None, - ) -> MutableMapping[str, Any]: - """ - Override this method to define the query parameters that should be set on an outgoing HTTP request given the inputs. - - E.g: you might want to define query parameters for paging if next_page_token is not None. - """ - return {} - - def request_headers( - self, - stream_state: Optional[Mapping[str, Any]], - stream_slice: Optional[Mapping[str, Any]] = None, - next_page_token: Optional[Mapping[str, Any]] = None, - ) -> Mapping[str, Any]: - """ - Override to return any non-auth headers. Authentication headers will overwrite any overlapping headers returned from this method. - """ - return {} - - def request_body_data( - self, - stream_state: Optional[Mapping[str, Any]], - stream_slice: Optional[Mapping[str, Any]] = None, - next_page_token: Optional[Mapping[str, Any]] = None, - ) -> Optional[Union[Mapping[str, Any], str]]: - """ - Override when creating POST/PUT/PATCH requests to populate the body of the request with a non-JSON payload. - - If returns a ready text that it will be sent as is. - If returns a dict that it will be converted to a urlencoded form. - E.g. {"key1": "value1", "key2": "value2"} => "key1=value1&key2=value2" - - At the same time only one of the 'request_body_data' and 'request_body_json' functions can be overridden. - """ - return None - - def request_body_json( - self, - stream_state: Optional[Mapping[str, Any]], - stream_slice: Optional[Mapping[str, Any]] = None, - next_page_token: Optional[Mapping[str, Any]] = None, - ) -> Optional[Mapping[str, Any]]: - """ - Override when creating POST/PUT/PATCH requests to populate the body of the request with a JSON payload. - - At the same time only one of the 'request_body_data' and 'request_body_json' functions can be overridden. - """ - return None - - def request_kwargs( - self, - stream_state: Optional[Mapping[str, Any]], - stream_slice: Optional[Mapping[str, Any]] = None, - next_page_token: Optional[Mapping[str, Any]] = None, - ) -> Mapping[str, Any]: - """ - Override to return a mapping of keyword arguments to be used when creating the HTTP request. - Any option listed in https://docs.python-requests.org/en/latest/api/#requests.adapters.BaseAdapter.send for can be returned from - this method. Note that these options do not conflict with request-level options such as headers, request params, etc.. - """ - return {} - @abstractmethod def parse_response( self, @@ -284,24 +156,6 @@ def error_message(self, response: requests.Response) -> str: """ return "" - def must_deduplicate_query_params(self) -> bool: - return False - - def deduplicate_query_params(self, url: str, params: Optional[Mapping[str, Any]]) -> Mapping[str, Any]: - """ - Remove query parameters from params mapping if they are already encoded in the URL. - :param url: URL with - :param params: - :return: - """ - if params is None: - params = {} - query_string = urllib.parse.urlparse(url).query - query_dict = {k: v[0] for k, v in urllib.parse.parse_qs(query_string).items()} - - duplicate_keys_with_same_value = {k for k in query_dict.keys() if str(params.get(k)) == str(query_dict[k])} - return {k: v for k, v in params.items() if k not in duplicate_keys_with_same_value} - def _create_prepared_request( self, path: str, @@ -329,10 +183,6 @@ def _create_prepared_request( return prepared_request - @classmethod - def _join_url(cls, url_base: str, path: str) -> str: - return urljoin(url_base, path) - def _send(self, request: requests.PreparedRequest, request_kwargs: Mapping[str, Any]) -> requests.Response: """ Wraps sending the request in rate limit and error handlers. diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 838c05729dbb..7e7b5c6322c0 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -18,7 +18,7 @@ from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.core_async import AsyncStream -from airbyte_cdk.sources.streams.http import HttpStream +from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH @@ -31,7 +31,7 @@ T = TypeVar("T") -class AsyncHttpStream(HttpStream, AsyncStream, ABC): +class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): """ Base abstract class for an Airbyte Stream using the HTTP protocol with asyncio. diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py new file mode 100644 index 000000000000..9cb9e8a9662f --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py @@ -0,0 +1,305 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import urllib +from abc import ABC, abstractmethod +from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Union +from urllib.parse import urljoin + +import requests +from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy +from airbyte_cdk.sources.streams.core import Stream, StreamData +from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy + +from .auth.core import HttpAuthenticator + +# list of all possible HTTP methods which can be used for sending of request bodies +BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") + + +class BaseHttpStream(Stream, ABC): + """ + Base abstract class for an Airbyte Stream using the HTTP protocol. Basic building block for users building an Airbyte source for a HTTP API. + """ + + source_defined_cursor = True # Most HTTP streams use a source defined cursor (i.e: the user can't configure it like on a SQL table) + page_size: Optional[int] = None # Use this variable to define page size for API http requests with pagination support + + @property + def cache_filename(self) -> str: + """ + Override if needed. Return the name of cache file + Note that if the environment variable REQUEST_CACHE_PATH is not set, the cache will be in-memory only. + """ + return f"{self.name}.sqlite" + + @property + def use_cache(self) -> bool: + """ + Override if needed. If True, all records will be cached. + Note that if the environment variable REQUEST_CACHE_PATH is not set, the cache will be in-memory only. + """ + return False + + @abstractmethod + def request_session(self) -> requests.Session: + """ + Session factory based on use_cache property and call rate limits (api_budget parameter) + :return: instance of request-based session + """ + + @abstractmethod + def clear_cache(self) -> None: + """ + Clear cached requests for current session, can be called any time + """ + + @property + @abstractmethod + def url_base(self) -> str: + """ + :return: URL base for the API endpoint e.g: if you wanted to hit https://myapi.com/v1/some_entity then this should return "https://myapi.com/v1/" + """ + + @property + def http_method(self) -> str: + """ + Override if needed. See get_request_data/get_request_json if using POST/PUT/PATCH. + """ + return "GET" + + @property + def raise_on_http_errors(self) -> bool: + """ + Override if needed. If set to False, allows opting-out of raising HTTP code exception. + """ + return True + + @property + def max_retries(self) -> Union[int, None]: + """ + Override if needed. Specifies maximum amount of retries for backoff policy. Return None for no limit. + """ + return 5 + + @property + def max_time(self) -> Union[int, None]: + """ + Override if needed. Specifies maximum total waiting time (in seconds) for backoff policy. Return None for no limit. + """ + return 60 * 10 + + @property + def retry_factor(self) -> float: + """ + Override if needed. Specifies factor for backoff policy. + """ + return 5 + + @property + @abstractmethod + def authenticator(self) -> HttpAuthenticator: + ... + + @property + def availability_strategy(self) -> Optional[AvailabilityStrategy]: + return HttpAvailabilityStrategy() + + @abstractmethod + def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + """ + Override this method to define a pagination strategy. + + The value returned from this method is passed to most other methods in this class. Use it to form a request e.g: set headers or query params. + + :return: The token for the next page from the input response object. Returning None means there are no more pages to read in this response. + """ + + @abstractmethod + def path( + self, + *, + stream_state: Optional[Mapping[str, Any]] = None, + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> str: + """ + Returns the URL path for the API endpoint e.g: if you wanted to hit https://myapi.com/v1/some_entity then this should return "some_entity" + """ + + def request_params( + self, + stream_state: Optional[Mapping[str, Any]], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> MutableMapping[str, Any]: + """ + Override this method to define the query parameters that should be set on an outgoing HTTP request given the inputs. + + E.g: you might want to define query parameters for paging if next_page_token is not None. + """ + return {} + + def request_headers( + self, + stream_state: Optional[Mapping[str, Any]], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Mapping[str, Any]: + """ + Override to return any non-auth headers. Authentication headers will overwrite any overlapping headers returned from this method. + """ + return {} + + def request_body_data( + self, + stream_state: Optional[Mapping[str, Any]], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Optional[Union[Mapping[str, Any], str]]: + """ + Override when creating POST/PUT/PATCH requests to populate the body of the request with a non-JSON payload. + + If returns a ready text that it will be sent as is. + If returns a dict that it will be converted to a urlencoded form. + E.g. {"key1": "value1", "key2": "value2"} => "key1=value1&key2=value2" + + At the same time only one of the 'request_body_data' and 'request_body_json' functions can be overridden. + """ + return None + + def request_body_json( + self, + stream_state: Optional[Mapping[str, Any]], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Optional[Mapping[str, Any]]: + """ + Override when creating POST/PUT/PATCH requests to populate the body of the request with a JSON payload. + + At the same time only one of the 'request_body_data' and 'request_body_json' functions can be overridden. + """ + return None + + def request_kwargs( + self, + stream_state: Optional[Mapping[str, Any]], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Mapping[str, Any]: + """ + Override to return a mapping of keyword arguments to be used when creating the HTTP request. + Any option listed in https://docs.python-requests.org/en/latest/api/#requests.adapters.BaseAdapter.send for can be returned from + this method. Note that these options do not conflict with request-level options such as headers, request params, etc.. + """ + return {} + + @abstractmethod + def parse_response( + self, + response: requests.Response, + *, + stream_state: Mapping[str, Any], + stream_slice: Optional[Mapping[str, Any]] = None, + next_page_token: Optional[Mapping[str, Any]] = None, + ) -> Iterable[Mapping[str, Any]]: + """ + Parses the raw response object into a list of records. + By default, this returns an iterable containing the input. Override to parse differently. + :param response: + :param stream_state: + :param stream_slice: + :param next_page_token: + :return: An iterable containing the parsed response + """ + + @abstractmethod + def should_retry(self, response: requests.Response) -> bool: + """ + Override to set different conditions for backoff based on the response from the server. + + By default, back off on the following HTTP response statuses: + - 429 (Too Many Requests) indicating rate limiting + - 500s to handle transient server errors + + Unexpected but transient exceptions (connection timeout, DNS resolution failed, etc..) are retried by default. + """ + + def backoff_time(self, response: requests.Response) -> Optional[float]: + """ + Override this method to dynamically determine backoff time e.g: by reading the X-Retry-After header. + + This method is called only if should_backoff() returns True for the input request. + + :param response: + :return how long to backoff in seconds. The return value may be a floating point number for subsecond precision. Returning None defers backoff + to the default backoff behavior (e.g using an exponential algorithm). + """ + return None + + def error_message(self, response: requests.Response) -> str: + """ + Override this method to specify a custom error message which can incorporate the HTTP response received + + :param response: The incoming HTTP response from the partner API + :return: + """ + return "" + + def must_deduplicate_query_params(self) -> bool: + return False + + def deduplicate_query_params(self, url: str, params: Optional[Mapping[str, Any]]) -> Mapping[str, Any]: + """ + Remove query parameters from params mapping if they are already encoded in the URL. + :param url: URL with + :param params: + :return: + """ + if params is None: + params = {} + query_string = urllib.parse.urlparse(url).query + query_dict = {k: v[0] for k, v in urllib.parse.parse_qs(query_string).items()} + + duplicate_keys_with_same_value = {k for k in query_dict.keys() if str(params.get(k)) == str(query_dict[k])} + return {k: v for k, v in params.items() if k not in duplicate_keys_with_same_value} + + @classmethod + def _join_url(cls, url_base: str, path: str) -> str: + return urljoin(url_base, path) + + @classmethod + @abstractmethod + def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: + """ + Parses the raw response object from a failed request into a user-friendly error message. + By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. + + :param response: + :return: A user-friendly message that indicates the cause of the error + """ + + @abstractmethod + def get_error_display_message(self, exception: BaseException) -> Optional[str]: + """ + Retrieves the user-friendly display message that corresponds to an exception. + This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. + + The default implementation of this method only handles HTTPErrors by passing the response to self.parse_response_error_message(). + The method should be overriden as needed to handle any additional exception types. + + :param exception: The exception that was raised + :return: A user-friendly message that indicates the cause of the error + """ + + @abstractmethod + def read_records( + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> Iterable[StreamData]: + ... From 8572328d5fb68535c550c68861a43a2d4cecc204 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 24 Dec 2023 13:50:06 -0500 Subject: [PATCH 22/74] Working: move session logic from Salesforce into AsyncHttpStream --- .../airbyte_cdk/sources/streams/http/http_async.py | 8 ++++++-- .../source-salesforce/source_salesforce/streams.py | 11 ----------- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 7e7b5c6322c0..be93e0018d9e 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -41,7 +41,6 @@ class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): def __init__(self, authenticator: HttpAuthenticator, api_budget: Optional[APIBudget] = None): self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) self._session: aiohttp.ClientSession = None - # self._session: aiohttp.ClientSession = self.request_session() # TODO: HttpStream handles other authentication codepaths, which may need to be added later self._authenticator = authenticator @@ -347,12 +346,17 @@ async def read_records( stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[StreamData]: - + self._session = await self._ensure_session() + assert not self._session.closed + # try: async for record in self._read_pages( lambda req, res, state, _slice: self.parse_response(res, stream_slice=_slice, stream_state=state), stream_slice, stream_state ): yield record + # finally: + # await self._session.close() + async def _read_pages( self, diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 8cabb6db6303..873324047efb 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -221,13 +221,8 @@ async def _read_pages( stream_slice: Mapping[str, Any] = None, stream_state: Mapping[str, Any] = None, ) -> None: - self._session = await self._ensure_session() - assert not self._session.closed - # try: async for record in self._do_read_pages(records_generator_fn, stream_slice, stream_state): yield record - # finally: - # await self._session.close() async def _do_read_pages( self, @@ -624,13 +619,8 @@ async def read_records( stream_slice: Mapping[str, Any] = None, stream_state: Mapping[str, Any] = None, ) -> Iterable[Mapping[str, Any]]: - self._session = await self._ensure_session() - assert not self._session.closed - # try: async for record in self._do_read_records(sync_mode, cursor_field, stream_slice, stream_state): yield record - # finally: - # await self._session.close() async def _do_read_records( self, @@ -686,7 +676,6 @@ def get_standard_instance(self) -> SalesforceStream: schema=self.schema, sobject_options=self.sobject_options, authenticator=self.authenticator, - session=self._session, ) new_cls: Type[SalesforceStream] = RestSalesforceStream if isinstance(self, BulkIncrementalSalesforceStream): From da1f8b8b8228da1794f809fd3026e4ab7976c3e9 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 24 Dec 2023 14:02:13 -0500 Subject: [PATCH 23/74] Working: simplify session logic --- .../sources/streams/http/http_async.py | 44 +++++++------------ .../source_salesforce/streams.py | 2 - 2 files changed, 17 insertions(+), 29 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index be93e0018d9e..96923ad87c17 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -280,7 +280,6 @@ async def _send_request(self, request: aiohttp.ClientRequest, request_kwargs: Ma """ if max_tries is not None: max_tries = max(0, max_tries) + 1 - assert not self._session.closed @default_backoff_handler(max_tries=max_tries, max_time=max_time, factor=self.retry_factor) @user_defined_backoff_handler(max_tries=max_tries, max_time=max_time) @@ -347,7 +346,6 @@ async def read_records( stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[StreamData]: self._session = await self._ensure_session() - assert not self._session.closed # try: async for record in self._read_pages( lambda req, res, state, _slice: self.parse_response(res, stream_slice=_slice, stream_state=state), stream_slice, @@ -366,30 +364,23 @@ async def _read_pages( stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[StreamData]: - self._session = await self._create_session() - assert self._session - assert not self._session.closed - try: - stream_state = stream_state or {} - pagination_complete = False - next_page_token = None - while not pagination_complete: - async def f(): - nonlocal next_page_token - assert not self._session.closed - request, response = await self._fetch_next_page(stream_slice, stream_state, next_page_token) - next_page_token = await self.next_page_token(response) - return request, response, next_page_token - - request, response, next_page_token = await f() - - async for record in records_generator_fn(request, response, stream_state, stream_slice): - yield record - - if not next_page_token: - pagination_complete = True - finally: - await self._session.close() + stream_state = stream_state or {} + pagination_complete = False + next_page_token = None + while not pagination_complete: + async def f(): + nonlocal next_page_token + request, response = await self._fetch_next_page(stream_slice, stream_state, next_page_token) + next_page_token = await self.next_page_token(response) + return request, response, next_page_token + + request, response, next_page_token = await f() + + async for record in records_generator_fn(request, response, stream_state, stream_slice): + yield record + + if not next_page_token: + pagination_complete = True async def _fetch_next_page( self, @@ -407,7 +398,6 @@ async def _fetch_next_page( ) request_kwargs = self.request_kwargs(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) - assert not self._session.closed response = await self._send_request(request, request_kwargs) return request, response diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 873324047efb..768c193d8b48 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -246,7 +246,6 @@ async def _do_read_pages( property_chunk = property_chunks[chunk_id] async def f(): - assert not self._session.closed request, response = await self._fetch_next_page_for_chunk( stream_slice, stream_state, property_chunk.next_page, property_chunk.properties ) @@ -303,7 +302,6 @@ async def _fetch_next_page_for_chunk( next_page_token: Mapping[str, Any] = None, property_chunk: Mapping[str, Any] = None, ) -> Tuple[aiohttp.ClientRequest, aiohttp.ClientResponse]: - assert not self._session.closed request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) request = self._create_prepared_request( path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), From b84069f3cadb67533ca38c2d59d8f66cd7e104f5 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 09:46:34 -0500 Subject: [PATCH 24/74] Manage sessions in the source --- .../airbyte_cdk/sources/abstract_source_async.py | 13 +++++++++++-- .../airbyte_cdk/sources/streams/http/http_async.py | 7 +------ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 67bdc358e1f6..903412301ef1 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -126,8 +126,17 @@ def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str yield record async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - tasks = [asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config)) for s in catalog.streams] - await asyncio.gather(*tasks) + tasks = [] + sessions = [] + try: + for s in catalog.streams: + stream = stream_instances.get(s.stream.name) + sessions.append(await stream._legacy_stream._ensure_session()) + tasks.append(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) + await asyncio.gather(*tasks) + finally: + for session in sessions: + await session.close() async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): print(f">>>>>>>>> _do_async_read_stream {configured_stream.stream.name}") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 96923ad87c17..4ed4035f2b63 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -241,7 +241,7 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st return response async def _ensure_session(self) -> aiohttp.ClientSession: - if self._session is None or self._session.closed: # TODO: why is the session closing? + if self._session is None: self._session = self.request_session() return self._session @@ -345,16 +345,11 @@ async def read_records( stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[StreamData]: - self._session = await self._ensure_session() - # try: async for record in self._read_pages( lambda req, res, state, _slice: self.parse_response(res, stream_slice=_slice, stream_state=state), stream_slice, stream_state ): yield record - # finally: - # await self._session.close() - async def _read_pages( self, From db25080351ae066d9da429ea8f9d8b0c6ca71b8b Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 11:59:09 -0500 Subject: [PATCH 25/74] Close sessions as the streams are finished processing --- .../sources/abstract_source_async.py | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 903412301ef1..527bc1c8c3fe 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -9,6 +9,7 @@ from threading import Thread from typing import Any, Callable, Dict, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, TypeVar, Union +import aiohttp from airbyte_cdk.models import ( AirbyteMessage, AirbyteStateMessage, @@ -43,6 +44,7 @@ def __init__(self, queue: Queue, sentinels: Dict[str, Sentinel], reader_fn: Call self.sentinels = sentinels self.reader_fn = reader_fn self.reader_args = args + self.sessions: Dict[str, aiohttp.ClientSession] = {} self.thread = Thread(target=self._start_reader_thread) self.thread.start() @@ -53,7 +55,9 @@ def _start_reader_thread(self): def __next__(self): item = self.queue.get() if isinstance(item, Sentinel): - print(f">>>>>>>>> Finished {item.name} stream") + session = self.sessions.pop(item.name) + if session: + session.close() self.sentinels.pop(item.name) # TODO: error handling? if not self.sentinels: self.thread.join() @@ -122,21 +126,17 @@ def read( def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): streams_in_progress = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams} - for record in SourceReader(self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config): + self.reader = SourceReader(self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) + for record in self.reader: yield record async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): tasks = [] - sessions = [] - try: - for s in catalog.streams: - stream = stream_instances.get(s.stream.name) - sessions.append(await stream._legacy_stream._ensure_session()) - tasks.append(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) - await asyncio.gather(*tasks) - finally: - for session in sessions: - await session.close() + for s in catalog.streams: + stream = stream_instances.get(s.stream.name) + self.reader.sessions[s.stream.name] = await stream._legacy_stream._ensure_session() + tasks.append(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) + await asyncio.gather(*tasks) async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): print(f">>>>>>>>> _do_async_read_stream {configured_stream.stream.name}") From d3e28754770179bfc02c273626f8755aa6489ff0 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 12:13:24 -0500 Subject: [PATCH 26/74] Working but awkward: handle session with SourceReader --- .../sources/abstract_source_async.py | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 527bc1c8c3fe..25853e5fc82b 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -39,7 +39,8 @@ def __init__(self, name: str): class SourceReader(Iterator): - def __init__(self, queue: Queue, sentinels: Dict[str, Sentinel], reader_fn: Callable, *args: Any): + def __init__(self, logger: logging.Logger, queue: Queue, sentinels: Dict[str, Sentinel], reader_fn: Callable, *args: Any): + self.logger = logger self.queue = queue self.sentinels = sentinels self.reader_fn = reader_fn @@ -55,10 +56,14 @@ def _start_reader_thread(self): def __next__(self): item = self.queue.get() if isinstance(item, Sentinel): - session = self.sessions.pop(item.name) - if session: - session.close() - self.sentinels.pop(item.name) # TODO: error handling? + # Sessions can only be closed once items in the stream have been dequeued + if session := self.sessions.pop(item.name): + loop = asyncio.get_event_loop() + loop.create_task(session.close()) + try: + self.sentinels.pop(item.name) + except KeyError: + raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") if not self.sentinels: self.thread.join() raise StopIteration @@ -126,7 +131,7 @@ def read( def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): streams_in_progress = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams} - self.reader = SourceReader(self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) + self.reader = SourceReader(logger, self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) for record in self.reader: yield record From a5e05f022688de3b7c6af0e5b752a5860e017064 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 13:24:40 -0500 Subject: [PATCH 27/74] Working: add back in availability check --- .../sources/abstract_source_async.py | 18 ++-- .../streams/concurrent/adapters_async.py | 8 +- .../airbyte_cdk/sources/streams/core_async.py | 14 ++- .../http/availability_strategy_async.py | 91 +++++++++++++++++++ .../airbyte_cdk/sources/streams/http/http.py | 4 + .../sources/streams/http/http_async.py | 5 + .../sources/streams/http/http_base.py | 6 -- .../streams/utils/stream_helper_async.py | 6 +- .../availability_strategy.py | 4 +- 9 files changed, 129 insertions(+), 27 deletions(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 25853e5fc82b..3a48dac41e49 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -114,8 +114,6 @@ def read( """Implements the Read operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/.""" logger.info(f"Starting syncing {self.name}") config, internal_config = split_config(config) - # TODO assert all streams exist in the connector - # get the streams once in case the connector needs to make any queries to generate them stream_instances = {s.name: s for s in self.streams(config)} state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) self._stream_to_instance_map = stream_instances @@ -155,17 +153,17 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s if not stream_instance: if not self.raise_exception_on_missing_stream: return - # raise KeyError( - # f"The stream {configured_stream.stream.name} no longer exists in the configuration. " - # f"Refresh the schema in replication settings and remove this stream from future sync attempts." - # ) + raise KeyError( + f"The stream {configured_stream.stream.name} no longer exists in the configuration. " + f"Refresh the schema in replication settings and remove this stream from future sync attempts." + ) try: # timer.start_event(f"Syncing stream {configured_stream.stream.name}") - # stream_is_available, reason = stream_instance.check_availability(logger, self) - # if not stream_is_available: - # logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") - # return + stream_is_available, reason = await stream_instance.check_availability(logger, self) + if not stream_is_available: + logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") + return logger.info(f"Marking stream {configured_stream.stream.name} as STARTED") self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.STARTED)) async for record in self._read_stream( diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py index 5e042be63ca8..2c3ca0bf7c63 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py @@ -215,14 +215,14 @@ def get_json_schema(self) -> Mapping[str, Any]: def supports_incremental(self) -> bool: return self._legacy_stream.supports_incremental - def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: + async def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: """ Verifies the stream is available. Delegates to the underlying AbstractStream and ignores the parameters :param logger: (ignored) :param source: (ignored) :return: """ - availability = self._abstract_stream.check_availability() + availability = await self._abstract_stream.check_availability() return availability.is_available(), availability.message() def get_error_display_message(self, exception: BaseException) -> Optional[str]: @@ -380,7 +380,7 @@ class AsyncAvailabilityStrategyFacade(AvailabilityStrategy): def __init__(self, abstract_availability_strategy: AbstractAvailabilityStrategy): self._abstract_availability_strategy = abstract_availability_strategy - def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, Optional[str]]: + async def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, Optional[str]]: """ Checks stream availability. @@ -392,7 +392,7 @@ def check_availability(self, stream: AsyncStream, logger: logging.Logger, source :return: A tuple of (boolean, str). If boolean is true, then the stream """ stream_availability = self._abstract_availability_strategy.check_availability(logger) - return stream_availability.is_available(), stream_availability.message() + return await stream_availability.is_available(), await stream_availability.message() class AsyncStreamAvailabilityStrategy(AbstractAvailabilityStrategy): diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py index ca8be5fea116..d3b8d44fc871 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -5,7 +5,7 @@ import logging from abc import ABC, abstractmethod -from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Tuple +from typing import TYPE_CHECKING, Any, Iterable, List, Mapping, MutableMapping, Optional, Tuple from airbyte_cdk.models import SyncMode from airbyte_cdk.models import Type as MessageType @@ -13,6 +13,9 @@ from airbyte_cdk.sources.utils.schema_helpers import InternalConfig from airbyte_cdk.sources.utils.slice_logger import SliceLogger +if TYPE_CHECKING: + from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy + class AsyncStream(Stream, ABC): """ @@ -113,6 +116,13 @@ async def stream_slices( """ return [None] + @property + def availability_strategy(self) -> Optional["AsyncHttpAvailabilityStrategy"]: + """ + :return: The AvailabilityStrategy used to check whether this stream is available. + """ + return None + async def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: """ Checks whether this stream is available. @@ -125,7 +135,7 @@ async def check_availability(self, logger: logging.Logger, source: Optional["Sou resolve the unavailability, if possible. """ if self.availability_strategy: - return self.availability_strategy.check_availability(self, logger, source) + return await self.availability_strategy.check_availability(self, logger, source) return True, None async def get_error_display_message(self, exception: BaseException) -> Optional[str]: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py new file mode 100644 index 000000000000..f467e67015a5 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py @@ -0,0 +1,91 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +import typing +from typing import Optional, Tuple + +from airbyte_cdk.sources.streams.core_async import AsyncStream +from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy +from airbyte_cdk.sources.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice +from requests import HTTPError + +if typing.TYPE_CHECKING: + from airbyte_cdk.sources import Source + + +class AsyncHttpAvailabilityStrategy(HttpAvailabilityStrategy): + async def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"]) -> Tuple[bool, Optional[str]]: + """ + Check stream availability by attempting to read the first record of the + stream. + + :param stream: stream + :param logger: source logger + :param source: (optional) source + :return: A tuple of (boolean, str). If boolean is true, then the stream + is available, and no str is required. Otherwise, the stream is unavailable + for some reason and the str should describe what went wrong and how to + resolve the unavailability, if possible. + """ + try: + # Some streams need a stream slice to read records (e.g. if they have a SubstreamPartitionRouter) + # Streams that don't need a stream slice will return `None` as their first stream slice. + stream_slice = await get_first_stream_slice(stream) + except StopIteration: + # If stream_slices has no `next()` item (Note - this is different from stream_slices returning [None]!) + # This can happen when a substream's `stream_slices` method does a `for record in parent_records: yield ` + # without accounting for the case in which the parent stream is empty. + reason = f"Cannot attempt to connect to stream {stream.name} - no stream slices were found, likely because the parent stream is empty." + return False, reason + except HTTPError as error: + is_available, reason = self.handle_http_error(stream, logger, source, error) + if not is_available: + reason = f"Unable to get slices for {stream.name} stream, because of error in parent stream. {reason}" + return is_available, reason + + try: + await get_first_record_for_slice(stream, stream_slice) + return True, None + except StopIteration: + logger.info(f"Successfully connected to stream {stream.name}, but got 0 records.") + return True, None + except HTTPError as error: + is_available, reason = self.handle_http_error(stream, logger, source, error) + if not is_available: + reason = f"Unable to read {stream.name} stream. {reason}" + return is_available, reason + + async def handle_http_error( + self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: HTTPError + ) -> Tuple[bool, Optional[str]]: + """ + Override this method to define error handling for various `HTTPError`s + that are raised while attempting to check a stream's availability. + + Checks whether an error's status_code is in a list of unavailable_error_codes, + and gets the associated reason for that error. + + :param stream: stream + :param logger: source logger + :param source: optional (source) + :param error: HTTPError raised while checking stream's availability. + :return: A tuple of (boolean, str). If boolean is true, then the stream + is available, and no str is required. Otherwise, the stream is unavailable + for some reason and the str should describe what went wrong and how to + resolve the unavailability, if possible. + """ + status_code = error.response.status_code + known_status_codes = self.reasons_for_unavailable_status_codes(stream, logger, source, error) + known_reason = known_status_codes.get(status_code) + if not known_reason: + # If the HTTPError is not in the dictionary of errors we know how to handle, don't except + raise error + + doc_ref = self._visit_docs_message(logger, source) + reason = f"The endpoint {error.response.url} returned {status_code}: {error.response.reason}. {known_reason}. {doc_ref} " + response_error_message = await stream.parse_response_error_message(error.response) + if response_error_message: + reason += response_error_message + return False, reason diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py index e588d679035d..4cb90971b326 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py @@ -81,6 +81,10 @@ def clear_cache(self) -> None: def authenticator(self) -> HttpAuthenticator: return self._authenticator + @property + def availability_strategy(self) -> Optional[AvailabilityStrategy]: + return HttpAvailabilityStrategy() + @abstractmethod def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: """ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 4ed4035f2b63..2c5ee00968a8 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -18,6 +18,7 @@ from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.core_async import AsyncStream +from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.utils.types import JsonType @@ -48,6 +49,10 @@ def __init__(self, authenticator: HttpAuthenticator, api_budget: Optional[APIBud def authenticator(self) -> HttpAuthenticator: return self._authenticator + @property + async def availability_strategy(self) -> Optional[AsyncHttpAvailabilityStrategy]: + return AsyncHttpAvailabilityStrategy() + def request_session(self) -> aiohttp.ClientSession: """ Session factory based on use_cache property and call rate limits (api_budget parameter) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py index 9cb9e8a9662f..2d59d3da0d69 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py @@ -9,9 +9,7 @@ import requests from airbyte_cdk.models import SyncMode -from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy from airbyte_cdk.sources.streams.core import Stream, StreamData -from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy from .auth.core import HttpAuthenticator @@ -103,10 +101,6 @@ def retry_factor(self) -> float: def authenticator(self) -> HttpAuthenticator: ... - @property - def availability_strategy(self) -> Optional[AvailabilityStrategy]: - return HttpAvailabilityStrategy() - @abstractmethod def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: """ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py index 6f9deb3b6cf8..7f7a325895b3 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py @@ -8,7 +8,7 @@ from airbyte_cdk.sources.streams.core import Stream, StreamData -def get_first_stream_slice(stream) -> Optional[Mapping[str, Any]]: +async def get_first_stream_slice(stream) -> Optional[Mapping[str, Any]]: """ Gets the first stream_slice from a given stream's stream_slices. :param stream: stream @@ -23,10 +23,10 @@ def get_first_stream_slice(stream) -> Optional[Mapping[str, Any]]: sync_mode=SyncMode.full_refresh, ) ) - return anext(slices) + return next(slices) -def get_first_record_for_slice(stream: Stream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: +async def get_first_record_for_slice(stream: Stream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: """ Gets the first record for a stream_slice of a stream. :param stream: stream diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py index 514727089c2a..988055d78f02 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py @@ -7,14 +7,14 @@ from typing import Optional, Tuple from airbyte_cdk.sources.streams import Stream -from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy +from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from requests import HTTPError, codes if typing.TYPE_CHECKING: from airbyte_cdk.sources import Source -class SalesforceAvailabilityStrategy(HttpAvailabilityStrategy): +class SalesforceAvailabilityStrategy(AsyncHttpAvailabilityStrategy): def handle_http_error( self, stream: Stream, logger: logging.Logger, source: Optional["Source"], error: HTTPError ) -> Tuple[bool, Optional[str]]: From 77fcbd670c3c209f26de535f619c16c43c360c2c Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 14:15:57 -0500 Subject: [PATCH 28/74] Working: re-add timer --- .../python/airbyte_cdk/sources/abstract_source_async.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 3a48dac41e49..5d6477dacd0c 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -159,7 +159,7 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s ) try: - # timer.start_event(f"Syncing stream {configured_stream.stream.name}") + timer.start_event(f"Syncing stream {configured_stream.stream.name}") stream_is_available, reason = await stream_instance.check_availability(logger, self) if not stream_is_available: logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") @@ -190,9 +190,9 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s raise AirbyteTracedException.from_exception(e, message=display_message) from e raise e finally: - # timer.finish_event() + timer.finish_event() logger.info(f"Finished syncing {configured_stream.stream.name}") - # logger.info(timer.report()) + logger.info(timer.report()) async def _read_stream( self, From 021fd30a2bc499b456a06587499a8b1a29af18c5 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 15:17:51 -0500 Subject: [PATCH 29/74] Working: async stream_slices --- .../streams/concurrent/adapters_async.py | 2 +- .../streams/utils/stream_helper_async.py | 24 ++++++++----------- .../source_salesforce/streams.py | 2 +- 3 files changed, 12 insertions(+), 16 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py index 2c3ca0bf7c63..45e2b760043c 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py @@ -369,7 +369,7 @@ def __init__( self._cursor = cursor async def generate(self) -> Iterable[Partition]: - for s in self._stream.stream_slices(sync_mode=self._sync_mode, cursor_field=self._cursor_field, stream_state=self._state): + async for s in self._stream.stream_slices(sync_mode=self._sync_mode, cursor_field=self._cursor_field, stream_state=self._state): yield AsyncStreamPartition( self._stream, copy.deepcopy(s), self.message_repository, self._sync_mode, self._cursor_field, self._state, self._cursor ) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py index 7f7a325895b3..3edf8fb64c3d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py @@ -5,28 +5,23 @@ from typing import Any, Mapping, Optional from airbyte_cdk.models import SyncMode -from airbyte_cdk.sources.streams.core import Stream, StreamData +from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.streams.core_async import AsyncStream -async def get_first_stream_slice(stream) -> Optional[Mapping[str, Any]]: +async def get_first_stream_slice(stream: AsyncStream) -> Optional[Mapping[str, Any]]: """ Gets the first stream_slice from a given stream's stream_slices. :param stream: stream :raises StopIteration: if there is no first slice to return (the stream_slices generator is empty) :return: first stream slice from 'stream_slices' generator (`None` is a valid stream slice) """ - # We wrap the return output of stream_slices() because some implementations return types that are iterable, - # but not iterators such as lists or tuples - slices = iter( - stream.stream_slices( - cursor_field=stream.cursor_field, - sync_mode=SyncMode.full_refresh, - ) - ) - return next(slices) + async for stream_slice in stream.stream_slices(cursor_field=stream.cursor_field, sync_mode=SyncMode.full_refresh): + return stream_slice + raise StopIteration(f"No slices in stream {stream.name}") -async def get_first_record_for_slice(stream: Stream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: +async def get_first_record_for_slice(stream: AsyncStream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: """ Gets the first record for a stream_slice of a stream. :param stream: stream @@ -36,5 +31,6 @@ async def get_first_record_for_slice(stream: Stream, stream_slice: Optional[Mapp """ # We wrap the return output of read_records() because some implementations return types that are iterable, # but not iterators such as lists or tuples - records_for_slice = stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slice) - return anext(records_for_slice) + async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slice): + return record + raise StopIteration(f"No records in stream {stream.name}") diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 768c193d8b48..09d499ca997e 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -708,7 +708,7 @@ def __init__(self, replication_key: str, **kwargs): super().__init__(**kwargs) self.replication_key = replication_key - def stream_slices( + async def stream_slices( self, *, sync_mode: SyncMode, cursor_field: List[str] = None, stream_state: Mapping[str, Any] = None ) -> Iterable[Optional[Mapping[str, Any]]]: start, end = (None, None) From c943190e7207269b3fdd7d93addf83142cf023e8 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 15:51:13 -0500 Subject: [PATCH 30/74] Working: allow a default concurrency limit to be set --- .../sources/abstract_source_async.py | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 5d6477dacd0c..cb319b572034 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -31,6 +31,7 @@ from airbyte_cdk.utils.traced_exception import AirbyteTracedException T = TypeVar("T") +DEFAULT_SESSION_LIMIT = 10000 class Sentinel: @@ -82,6 +83,7 @@ class AsyncAbstractSource(AbstractSource, ABC): def __init__(self): super().__init__() self.queue = Queue(10000) + self.session_limit = DEFAULT_SESSION_LIMIT @abstractmethod def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: @@ -134,12 +136,20 @@ def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str yield record async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - tasks = [] - for s in catalog.streams: - stream = stream_instances.get(s.stream.name) - self.reader.sessions[s.stream.name] = await stream._legacy_stream._ensure_session() - tasks.append(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) - await asyncio.gather(*tasks) + pending_tasks = set() + n_started, n_streams = 0, len(catalog.streams) + streams_iterator = iter(catalog.streams) + + while pending_tasks or n_started < n_streams: + while len(pending_tasks) <= self.session_limit and (s := next(streams_iterator, None)): + if s is None: + break + stream = stream_instances.get(s.stream.name) + self.reader.sessions[s.stream.name] = await stream._legacy_stream._ensure_session() + pending_tasks.add(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) + n_started += 1 + + _, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): print(f">>>>>>>>> _do_async_read_stream {configured_stream.stream.name}") From 34a90e3e216825d4a68b98ad4668c8acce85052b Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 26 Dec 2023 16:31:36 -0500 Subject: [PATCH 31/74] Working: don't use adapters.py --- .../python/airbyte_cdk/sources/abstract_source_async.py | 4 ++-- airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py | 5 ++--- .../connectors/source-salesforce/source_salesforce/source.py | 2 -- .../source-salesforce/source_salesforce/streams.py | 4 ++-- 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index cb319b572034..de4f859ecef2 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -145,7 +145,7 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance if s is None: break stream = stream_instances.get(s.stream.name) - self.reader.sessions[s.stream.name] = await stream._legacy_stream._ensure_session() + self.reader.sessions[s.stream.name] = await stream._ensure_session() pending_tasks.add(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) n_started += 1 @@ -195,7 +195,7 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) - display_message = stream_instance.get_error_display_message(e) + display_message = await stream_instance.get_error_display_message(e) if display_message: raise AirbyteTracedException.from_exception(e, message=display_message) from e raise e diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py index d3b8d44fc871..91f07339d394 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -28,9 +28,8 @@ async def read_full_refresh( logger: logging.Logger, slice_logger: SliceLogger, ) -> Iterable[StreamData]: - slices = self.stream_slices(sync_mode=SyncMode.full_refresh, cursor_field=cursor_field) - logger.debug(f"Processing stream slices for {self.name} (sync_mode: full_refresh)", extra={"stream_slices": slices}) - for _slice in slices: + async for _slice in self.stream_slices(sync_mode=SyncMode.full_refresh, cursor_field=cursor_field): + logger.debug(f"Processing stream slices for {self.name} (sync_mode: full_refresh)", extra={"stream_slice": _slice}) if slice_logger.should_log_slice_message(logger): yield slice_logger.create_slice_log_message(_slice) async for record in self.read_records( diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index 15bca7a60335..e8b2c5ffe62f 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -14,8 +14,6 @@ from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import InMemoryMessageRepository from airbyte_cdk.sources.streams import Stream -from airbyte_cdk.sources.streams.concurrent.adapters_async import AsyncStreamFacade -from airbyte_cdk.sources.streams.concurrent.cursor import NoopCursor from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator from airbyte_cdk.sources.utils.schema_helpers import InternalConfig from airbyte_cdk.utils.traced_exception import AirbyteTracedException diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 09d499ca997e..4cba6e2f79b7 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -106,10 +106,10 @@ def get_json_schema(self) -> Mapping[str, Any]: self.schema = self.sf_api.generate_schema(self.name) return self.schema - def get_error_display_message(self, exception: BaseException) -> Optional[str]: + async def get_error_display_message(self, exception: BaseException) -> Optional[str]: if isinstance(exception, exceptions.ConnectionError): return f"After {self.max_retries} retries the connector has failed with a network error. It looks like Salesforce API experienced temporary instability, please try again later." - return super().get_error_display_message(exception) + return await super().get_error_display_message(exception) class PropertyChunk: From 5fcbd8d622fc5abc783c79fb6378d3303e78da4b Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 10:40:59 -0500 Subject: [PATCH 32/74] Passing: test_streams_core_async.py --- .../airbyte_cdk/sources/streams/__init__.py | 3 +- .../streams/concurrent/adapters_async.py | 2 +- .../http/availability_strategy_async.py | 2 +- .../sources/streams/http/http_async.py | 2 +- .../streams/utils/stream_helper_async.py | 2 +- .../streams/test_streams_core_async.py | 184 ++++++++++++++++++ 6 files changed, 190 insertions(+), 5 deletions(-) create mode 100644 airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py index 9326fd1bdca7..f50ec9119bd9 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py @@ -4,5 +4,6 @@ # Initialize Streams Package from .core import IncrementalMixin, Stream +from .core_async import AsyncStream -__all__ = ["IncrementalMixin", "Stream"] +__all__ = ["IncrementalMixin", "Stream", "AsyncStream"] diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py index 45e2b760043c..8993b50e90dc 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py @@ -12,7 +12,7 @@ from airbyte_cdk.sources import AbstractSource, Source from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import MessageRepository -from airbyte_cdk.sources.streams.core_async import AsyncStream +from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream from airbyte_cdk.sources.streams.concurrent.availability_strategy import ( diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py index f467e67015a5..1f724af4db15 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py @@ -6,7 +6,7 @@ import typing from typing import Optional, Tuple -from airbyte_cdk.sources.streams.core_async import AsyncStream +from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy from airbyte_cdk.sources.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice from requests import HTTPError diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 2c5ee00968a8..ad9745dd93d5 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -14,10 +14,10 @@ import aiohttp_client_cache from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE +from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData -from airbyte_cdk.sources.streams.core_async import AsyncStream from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py index 3edf8fb64c3d..efdb50922a59 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py @@ -5,8 +5,8 @@ from typing import Any, Mapping, Optional from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.core import StreamData -from airbyte_cdk.sources.streams.core_async import AsyncStream async def get_first_stream_slice(stream: AsyncStream) -> Optional[Mapping[str, Any]]: diff --git a/airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py b/airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py new file mode 100644 index 000000000000..08bfd1da428b --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py @@ -0,0 +1,184 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + + +from typing import Any, Iterable, List, Mapping +from unittest import mock + +import pytest +from airbyte_cdk.models import AirbyteStream, SyncMode +from airbyte_cdk.sources.streams import AsyncStream + + +class StreamStubFullRefresh(AsyncStream): + """ + Stub full refresh class to assist with testing. + """ + + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: List[str] = None, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ) -> Iterable[Mapping[str, Any]]: + pass + + primary_key = None + + +def test_as_airbyte_stream_full_refresh(mocker): + """ + Should return an full refresh AirbyteStream with information matching the + provided Stream interface. + """ + test_stream = StreamStubFullRefresh() + + mocker.patch.object(StreamStubFullRefresh, "get_json_schema", return_value={}) + airbyte_stream = test_stream.as_airbyte_stream() + + exp = AirbyteStream(name="stream_stub_full_refresh", json_schema={}, supported_sync_modes=[SyncMode.full_refresh]) + assert exp == airbyte_stream + + +class StreamStubIncremental(AsyncStream): + """ + Stub full incremental class to assist with testing. + """ + + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: List[str] = None, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ) -> Iterable[Mapping[str, Any]]: + pass + + cursor_field = "test_cursor" + primary_key = "primary_key" + namespace = "test_namespace" + + +class StreamStubIncrementalEmptyNamespace(AsyncStream): + """ + Stub full incremental class, with empty namespace, to assist with testing. + """ + + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: List[str] = None, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ) -> Iterable[Mapping[str, Any]]: + pass + + cursor_field = "test_cursor" + primary_key = "primary_key" + namespace = "" + + +def test_as_airbyte_stream_incremental(mocker): + """ + Should return an incremental refresh AirbyteStream with information matching + the provided Stream interface. + """ + test_stream = StreamStubIncremental() + + mocker.patch.object(StreamStubIncremental, "get_json_schema", return_value={}) + airbyte_stream = test_stream.as_airbyte_stream() + + exp = AirbyteStream( + name="stream_stub_incremental", + namespace="test_namespace", + json_schema={}, + supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], + default_cursor_field=["test_cursor"], + source_defined_cursor=True, + source_defined_primary_key=[["primary_key"]], + ) + assert exp == airbyte_stream + + +def test_supports_incremental_cursor_set(): + """ + Should return true if cursor is set. + """ + test_stream = StreamStubIncremental() + test_stream.cursor_field = "test_cursor" + + assert test_stream.supports_incremental + + +def test_supports_incremental_cursor_not_set(): + """ + Should return false if cursor is not. + """ + test_stream = StreamStubFullRefresh() + + assert not test_stream.supports_incremental + + +def test_namespace_set(): + """ + Should allow namespace property to be set. + """ + test_stream = StreamStubIncremental() + + assert test_stream.namespace == "test_namespace" + + +def test_namespace_set_to_empty_string(mocker): + """ + Should not set namespace property if equal to empty string. + """ + test_stream = StreamStubIncremental() + + mocker.patch.object(StreamStubIncremental, "get_json_schema", return_value={}) + mocker.patch.object(StreamStubIncremental, "namespace", "") + + airbyte_stream = test_stream.as_airbyte_stream() + + exp = AirbyteStream( + name="stream_stub_incremental", + json_schema={}, + supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], + default_cursor_field=["test_cursor"], + source_defined_cursor=True, + source_defined_primary_key=[["primary_key"]], + namespace=None, + ) + assert exp == airbyte_stream + + +def test_namespace_not_set(): + """ + Should be equal to unset value of None. + """ + test_stream = StreamStubFullRefresh() + + assert test_stream.namespace is None + + +@pytest.mark.parametrize( + "test_input, expected", + [("key", [["key"]]), (["key1", "key2"], [["key1"], ["key2"]]), ([["key1", "key2"], ["key3"]], [["key1", "key2"], ["key3"]])], +) +def test_wrapped_primary_key_various_argument(test_input, expected): + """ + Should always wrap primary key into list of lists. + """ + + wrapped = AsyncStream._wrapped_primary_key(test_input) + + assert wrapped == expected + + +@mock.patch("airbyte_cdk.sources.utils.schema_helpers.ResourceSchemaLoader.get_schema") +def test_get_json_schema_is_cached(mocked_method): + stream = StreamStubFullRefresh() + for i in range(5): + stream.get_json_schema() + assert mocked_method.call_count == 1 From 90e61de6ab122b95cb19a78884ab3d49534e431c Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 12:04:00 -0500 Subject: [PATCH 33/74] Working: cleanup --- .../sources/abstract_source_async.py | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index de4f859ecef2..724706b1ae3b 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -21,7 +21,7 @@ from airbyte_cdk.models import Type as MessageType from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.abstract_source import AbstractSource -from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.http import HttpStream from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message @@ -99,7 +99,7 @@ def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> """ @abstractmethod - def streams(self, config: Mapping[str, Any]) -> List[Stream]: + def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: """ :param config: The user-provided configuration as specified by the source's spec. Any stream construction related operation should happen here. @@ -129,37 +129,37 @@ def read( print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") logger.info(f"Finished syncing {self.name}") - def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): streams_in_progress = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams} self.reader = SourceReader(logger, self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) for record in self.reader: yield record - async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): pending_tasks = set() n_started, n_streams = 0, len(catalog.streams) streams_iterator = iter(catalog.streams) while pending_tasks or n_started < n_streams: - while len(pending_tasks) <= self.session_limit and (s := next(streams_iterator, None)): - if s is None: + while len(pending_tasks) <= self.session_limit and (configured_stream := next(streams_iterator, None)): + if configured_stream is None: break - stream = stream_instances.get(s.stream.name) - self.reader.sessions[s.stream.name] = await stream._ensure_session() - pending_tasks.add(asyncio.create_task(self._do_async_read_stream(s, stream_instances, timer, logger, state_manager, internal_config))) + stream_instance = stream_instances.get("Account") + stream = stream_instances.get(configured_stream.stream.name) + self.reader.sessions[configured_stream.stream.name] = await stream._ensure_session() + pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) n_started += 1 _, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) - async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): print(f">>>>>>>>> _do_async_read_stream {configured_stream.stream.name}") try: - await self._async_read_stream(configured_stream, stream_instances, timer, logger, state_manager, internal_config) + await self._async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config) finally: self.queue.put(Sentinel(configured_stream.stream.name)) - async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instances: Dict[str, Stream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - stream_instance = stream_instances.get("Account") + async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): if not stream_instance: if not self.raise_exception_on_missing_stream: return @@ -207,7 +207,7 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s async def _read_stream( self, logger: logging.Logger, - stream_instance: Stream, + stream_instance: AsyncStream, configured_stream: ConfiguredAirbyteStream, state_manager: ConnectorStateManager, internal_config: InternalConfig, @@ -256,7 +256,7 @@ async def _read_stream( async def _read_incremental( self, logger: logging.Logger, - stream_instance: Stream, + stream_instance: AsyncStream, configured_stream: ConfiguredAirbyteStream, state_manager: ConnectorStateManager, internal_config: InternalConfig, @@ -296,7 +296,7 @@ def _emit_queued_messages(self) -> Iterable[AirbyteMessage]: async def _read_full_refresh( self, logger: logging.Logger, - stream_instance: Stream, + stream_instance: AsyncStream, configured_stream: ConfiguredAirbyteStream, internal_config: InternalConfig, ) -> Iterator[AirbyteMessage]: @@ -309,7 +309,7 @@ async def _read_full_refresh( if internal_config.is_limit_reached(total_records_counter): return - def _get_message(self, record_data_or_message: Union[StreamData, AirbyteMessage], stream: Stream) -> AirbyteMessage: + def _get_message(self, record_data_or_message: Union[StreamData, AirbyteMessage], stream: AsyncStream) -> AirbyteMessage: """ Converts the input to an AirbyteMessage if it is a StreamData. Returns the input as is if it is already an AirbyteMessage """ From 61cb63aa570ee9134eac7308dacb6cfcfb587f22 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 12:07:39 -0500 Subject: [PATCH 34/74] Working: check for streams before starting the reads --- .../sources/abstract_source_async.py | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 724706b1ae3b..5c281bc8363d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -131,10 +131,22 @@ def read( def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): streams_in_progress = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams} + self._assert_streams(catalog, stream_instances) self.reader = SourceReader(logger, self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) for record in self.reader: yield record + def _assert_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream]): + for configured_stream in catalog.streams: + stream_instance = stream_instances.get(configured_stream.stream.name) + if not stream_instance: + if not self.raise_exception_on_missing_stream: + return + raise KeyError( + f"The stream {configured_stream.stream.name} no longer exists in the configuration. " + f"Refresh the schema in replication settings and remove this stream from future sync attempts." + ) + async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): pending_tasks = set() n_started, n_streams = 0, len(catalog.streams) @@ -144,7 +156,7 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance while len(pending_tasks) <= self.session_limit and (configured_stream := next(streams_iterator, None)): if configured_stream is None: break - stream_instance = stream_instances.get("Account") + stream_instance = stream_instances.get("Account") # TODO stream = stream_instances.get(configured_stream.stream.name) self.reader.sessions[configured_stream.stream.name] = await stream._ensure_session() pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) @@ -153,21 +165,12 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance _, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - print(f">>>>>>>>> _do_async_read_stream {configured_stream.stream.name}") try: await self._async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config) finally: self.queue.put(Sentinel(configured_stream.stream.name)) async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - if not stream_instance: - if not self.raise_exception_on_missing_stream: - return - raise KeyError( - f"The stream {configured_stream.stream.name} no longer exists in the configuration. " - f"Refresh the schema in replication settings and remove this stream from future sync attempts." - ) - try: timer.start_event(f"Syncing stream {configured_stream.stream.name}") stream_is_available, reason = await stream_instance.check_availability(logger, self) From eed55f68af53f76bd35efba2ae3bdb77ce6d5614 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 12:28:51 -0500 Subject: [PATCH 35/74] Working: only sync stream if it's in the catalog --- .../python/airbyte_cdk/sources/abstract_source_async.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 5c281bc8363d..a9e79a36bda4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -119,6 +119,7 @@ def read( stream_instances = {s.name: s for s in self.streams(config)} state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) self._stream_to_instance_map = stream_instances + self._assert_streams(catalog, stream_instances) n_records = 0 with create_timer(self.name) as timer: @@ -130,9 +131,10 @@ def read( logger.info(f"Finished syncing {self.name}") def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - streams_in_progress = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams} - self._assert_streams(catalog, stream_instances) - self.reader = SourceReader(logger, self.queue, streams_in_progress, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) + streams_in_progress_sentinels = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams if s.stream.name in stream_instances} + if not streams_in_progress_sentinels: + return + self.reader = SourceReader(logger, self.queue, streams_in_progress_sentinels, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) for record in self.reader: yield record From 98e5e0dcd1b0c3ccac141f1d0255130fb50d78e0 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 13:27:44 -0500 Subject: [PATCH 36/74] Working: move ensure_session onto AsyncStream --- .../python/airbyte_cdk/sources/abstract_source_async.py | 3 ++- .../python/airbyte_cdk/sources/streams/core_async.py | 6 ++++++ .../python/airbyte_cdk/sources/streams/http/http_async.py | 2 +- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index a9e79a36bda4..4cc83bac25ab 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -159,8 +159,9 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance if configured_stream is None: break stream_instance = stream_instances.get("Account") # TODO + # stream_instance = stream_instances.get(configured_stream.stream.name) stream = stream_instances.get(configured_stream.stream.name) - self.reader.sessions[configured_stream.stream.name] = await stream._ensure_session() + self.reader.sessions[configured_stream.stream.name] = await stream.ensure_session() pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) n_started += 1 diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py index 91f07339d394..ac80e08a9592 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -115,6 +115,12 @@ async def stream_slices( """ return [None] + async def ensure_session(self, *args: Any, **kwargs: Any) -> Any: + """ + Override to define a session object on the stream. + """ + pass + @property def availability_strategy(self) -> Optional["AsyncHttpAvailabilityStrategy"]: """ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index ad9745dd93d5..8bbd2d7929bd 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -245,7 +245,7 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st raise exc return response - async def _ensure_session(self) -> aiohttp.ClientSession: + async def ensure_session(self) -> aiohttp.ClientSession: if self._session is None: self._session = self.request_session() return self._session From de25233619111dc661fbf0fb546e02ecd355671a Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 17:32:55 -0500 Subject: [PATCH 37/74] Working: handle errors from reader thread --- .../python/airbyte_cdk/sources/__init__.py | 3 +- .../sources/abstract_source_async.py | 31 ++++++++++++++++--- .../airbyte_cdk/sources/streams/core_async.py | 2 +- 3 files changed, 29 insertions(+), 7 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/__init__.py index 396513539fcd..631c2b0762fa 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/__init__.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/__init__.py @@ -5,6 +5,7 @@ import dpath.options from .abstract_source import AbstractSource +from .abstract_source_async import AsyncAbstractSource from .config import BaseConfig from .source import Source @@ -19,4 +20,4 @@ # this will not be thread-safe. dpath.options.ALLOW_EMPTY_STRING_KEYS = True -__all__ = ["AbstractSource", "BaseConfig", "Source"] +__all__ = ["AbstractSource", "AsyncAbstractSource", "BaseConfig", "Source"] diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 4cc83bac25ab..65da0f46f780 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -4,9 +4,10 @@ import asyncio import logging +import traceback from abc import ABC, abstractmethod from queue import Queue -from threading import Thread +from threading import Lock, Thread from typing import Any, Callable, Dict, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, TypeVar, Union import aiohttp @@ -32,6 +33,7 @@ T = TypeVar("T") DEFAULT_SESSION_LIMIT = 10000 +DEFAULT_TIMEOUT = None class Sentinel: @@ -47,17 +49,28 @@ def __init__(self, logger: logging.Logger, queue: Queue, sentinels: Dict[str, Se self.reader_fn = reader_fn self.reader_args = args self.sessions: Dict[str, aiohttp.ClientSession] = {} + self.error = None + self.error_lock = Lock() self.thread = Thread(target=self._start_reader_thread) self.thread.start() def _start_reader_thread(self): - asyncio.run(self.reader_fn(*self.reader_args)) + try: + asyncio.run(self.reader_fn(*self.reader_args)) + except Exception as e: + with self.error_lock: + self.error = (e, traceback.format_exc()) def __next__(self): - item = self.queue.get() + with self.error_lock: + if self.error: + exception, traceback_str = self.error + self.logger.error(f"An error occurred in the async thread: {traceback_str}") + raise exception + item = self.queue.get(timeout=DEFAULT_TIMEOUT) if isinstance(item, Sentinel): - # Sessions can only be closed once items in the stream have been dequeued + # Sessions can only be closed once items in the stream have all been dequeued if session := self.sessions.pop(item.name): loop = asyncio.get_event_loop() loop.create_task(session.close()) @@ -165,7 +178,13 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) n_started += 1 - _, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) + done, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) + + for task in done: + if task.exception(): + for remaining_task in pending_tasks: + remaining_task.cancel() + raise task.exception() async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): try: @@ -196,6 +215,8 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) raise e except Exception as e: + with self.reader.error_lock: + self.reader.error = (e, traceback.format_exc()) for message in self._emit_queued_messages(): self.queue.put(message) logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py index ac80e08a9592..d1e4c287bce7 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -113,7 +113,7 @@ async def stream_slices( :param stream_state: :return: """ - return [None] + yield [None] async def ensure_session(self, *args: Any, **kwargs: Any) -> Any: """ From 0a0d0a7926718fd71aa1f0591e7bf3b9c550ea1f Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 18:40:04 -0500 Subject: [PATCH 38/74] Working: some error handling fixes --- .../sources/abstract_source_async.py | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 65da0f46f780..507086b2fff4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -58,7 +58,7 @@ def __init__(self, logger: logging.Logger, queue: Queue, sentinels: Dict[str, Se def _start_reader_thread(self): try: asyncio.run(self.reader_fn(*self.reader_args)) - except Exception as e: + except Exception as e: # TODO: is this definitely needed? with self.error_lock: self.error = (e, traceback.format_exc()) @@ -71,7 +71,7 @@ def __next__(self): item = self.queue.get(timeout=DEFAULT_TIMEOUT) if isinstance(item, Sentinel): # Sessions can only be closed once items in the stream have all been dequeued - if session := self.sessions.pop(item.name): + if session := self.sessions.pop(item.name, None): loop = asyncio.get_event_loop() loop.create_task(session.close()) try: @@ -171,8 +171,8 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance while len(pending_tasks) <= self.session_limit and (configured_stream := next(streams_iterator, None)): if configured_stream is None: break - stream_instance = stream_instances.get("Account") # TODO - # stream_instance = stream_instances.get(configured_stream.stream.name) + # stream_instance = stream_instances.get("Account") # TODO + stream_instance = stream_instances.get(configured_stream.stream.name) stream = stream_instances.get(configured_stream.stream.name) self.reader.sessions[configured_stream.stream.name] = await stream.ensure_session() pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) @@ -213,10 +213,10 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) except AirbyteTracedException as e: self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) - raise e - except Exception as e: with self.reader.error_lock: self.reader.error = (e, traceback.format_exc()) + raise e + except Exception as e: for message in self._emit_queued_messages(): self.queue.put(message) logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") @@ -224,7 +224,12 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) display_message = await stream_instance.get_error_display_message(e) if display_message: - raise AirbyteTracedException.from_exception(e, message=display_message) from e + exc = AirbyteTracedException.from_exception(e, message=display_message) + with self.reader.error_lock: + self.reader.error = (exc, traceback.format_exc()) + raise exc from e + with self.reader.error_lock: + self.reader.error = (e, traceback.format_exc()) raise e finally: timer.finish_event() From c2b726578ab6cefecce2ef7dd3b963ce4ee56d2a Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 19:44:10 -0500 Subject: [PATCH 39/74] Passing: test_abstract_source_async.py --- .../airbyte_cdk/sources/streams/core_async.py | 6 +- .../sources/test_abstract_source_async.py | 1217 +++++++++++++++++ 2 files changed, 1220 insertions(+), 3 deletions(-) create mode 100644 airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py index d1e4c287bce7..442f67ab59d8 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py @@ -58,7 +58,7 @@ async def read_incremental( # type: ignore # ignoring typing for ConnectorStat has_slices = False record_counter = 0 - for _slice in slices: + async for _slice in slices: has_slices = True if slice_logger.should_log_slice_message(logger): yield slice_logger.create_slice_log_message(_slice) @@ -68,7 +68,7 @@ async def read_incremental( # type: ignore # ignoring typing for ConnectorStat stream_state=stream_state, cursor_field=cursor_field or None, ) - for record_data_or_message in records: + async for record_data_or_message in records: yield record_data_or_message if isinstance(record_data_or_message, Mapping) or ( hasattr(record_data_or_message, "type") and record_data_or_message.type == MessageType.RECORD @@ -113,7 +113,7 @@ async def stream_slices( :param stream_state: :return: """ - yield [None] + yield None async def ensure_session(self, *args: Any, **kwargs: Any) -> Any: """ diff --git a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py b/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py new file mode 100644 index 000000000000..d9f1e82b3495 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py @@ -0,0 +1,1217 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import copy +import datetime +import logging +from collections import defaultdict +from typing import Any, Callable, Dict, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Union +from unittest.mock import Mock, call +from unittest.mock import AsyncMock + +import pytest +from airbyte_cdk.models import ( + AirbyteCatalog, + AirbyteConnectionStatus, + AirbyteLogMessage, + AirbyteMessage, + AirbyteRecordMessage, + AirbyteStateBlob, + AirbyteStateMessage, + AirbyteStateType, + AirbyteStream, + AirbyteStreamState, + AirbyteStreamStatus, + AirbyteStreamStatusTraceMessage, + AirbyteTraceMessage, + ConfiguredAirbyteCatalog, + ConfiguredAirbyteStream, + DestinationSyncMode, + Level, + Status, + StreamDescriptor, + SyncMode, + TraceType, +) +from airbyte_cdk.models import Type +from airbyte_cdk.models import Type as MessageType +from airbyte_cdk.sources import AsyncAbstractSource +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.message import MessageRepository +from airbyte_cdk.sources.streams import AsyncStream, IncrementalMixin +from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message +from airbyte_cdk.utils.traced_exception import AirbyteTracedException +from pytest import fixture + +logger = logging.getLogger("airbyte") + + +class MockSource(AsyncAbstractSource): + def __init__( + self, + check_lambda: Callable[[], Tuple[bool, Optional[Any]]] = None, + streams: List[AsyncStream] = None, + per_stream: bool = True, + message_repository: MessageRepository = None, + exception_on_missing_stream: bool = True, + ): + self._streams = streams + self.check_lambda = check_lambda + self.per_stream = per_stream + self.exception_on_missing_stream = exception_on_missing_stream + self._message_repository = message_repository + super().__init__() + + def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + if self.check_lambda: + return self.check_lambda() + return False, "Missing callable." + + def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: + if not self._streams: + raise Exception("Stream is not set") + return self._streams + + @property + def raise_exception_on_missing_stream(self) -> bool: + return self.exception_on_missing_stream + + @property + def per_stream_state_enabled(self) -> bool: + return self.per_stream + + @property + def message_repository(self): + return self._message_repository + + +class StreamNoStateMethod(AsyncStream): + name = "managers" + primary_key = None + + async def read_records(self, *args, **kwargs) -> Iterable[Mapping[str, Any]]: + yield {} + + +class MockStreamOverridesStateMethod(AsyncStream, IncrementalMixin): + name = "teams" + primary_key = None + cursor_field = "updated_at" + _cursor_value = "" + start_date = "1984-12-12" + + async def read_records(self, *args, **kwargs) -> Iterable[Mapping[str, Any]]: + yield {} + + @property + def state(self) -> MutableMapping[str, Any]: + return {self.cursor_field: self._cursor_value} if self._cursor_value else {} + + @state.setter + def state(self, value: MutableMapping[str, Any]): + self._cursor_value = value.get(self.cursor_field, self.start_date) + + +MESSAGE_FROM_REPOSITORY = Mock() + + +@fixture +def message_repository(): + message_repository = Mock(spec=MessageRepository) + message_repository.consume_queue.return_value = [message for message in [MESSAGE_FROM_REPOSITORY]] + return message_repository + + +def test_successful_check(): + """Tests that if a source returns TRUE for the connection check the appropriate connectionStatus success message is returned""" + expected = AirbyteConnectionStatus(status=Status.SUCCEEDED) + assert expected == MockSource(check_lambda=lambda: (True, None)).check(logger, {}) + + +def test_failed_check(): + """Tests that if a source returns FALSE for the connection check the appropriate connectionStatus failure message is returned""" + expected = AirbyteConnectionStatus(status=Status.FAILED, message="'womp womp'") + assert expected == MockSource(check_lambda=lambda: (False, "womp womp")).check(logger, {}) + + +def test_raising_check(mocker): + """Tests that if a source raises an unexpected exception the appropriate connectionStatus failure message is returned.""" + check_lambda = mocker.Mock(side_effect=BaseException("this should fail")) + with pytest.raises(BaseException): + MockSource(check_lambda=check_lambda).check(logger, {}) + + +class MockStream(AsyncStream): + def __init__( + self, + inputs_and_mocked_outputs: List[Tuple[Mapping[str, Any], Iterable[Mapping[str, Any]]]] = None, + name: str = None, + ): + self._inputs_and_mocked_outputs = inputs_and_mocked_outputs + self._name = name + + @property + def name(self): + return self._name + + async def read_records(self, **kwargs) -> Iterable[Mapping[str, Any]]: # type: ignore + # Remove None values + kwargs = {k: v for k, v in kwargs.items() if v is not None} + output_supplied = False + if self._inputs_and_mocked_outputs: + for _input, output in self._inputs_and_mocked_outputs: + if kwargs == _input: + if isinstance(output, list): + for item in output: + yield item + output_supplied = True + else: + output_supplied = True + yield output + + if not output_supplied: + raise Exception(f"No mocked output supplied for input: {kwargs}. Mocked inputs/outputs: {self._inputs_and_mocked_outputs}") + + @property + def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: + return "pk" + + +class MockStreamWithState(MockStream): + cursor_field = "cursor" + + def __init__(self, inputs_and_mocked_outputs: List[Tuple[Mapping[str, Any], Iterable[Mapping[str, Any]]]], name: str, state=None): + super().__init__(inputs_and_mocked_outputs, name) + self._state = state + + @property + def state(self): + return self._state + + @state.setter + def state(self, value): + pass + + +class MockStreamEmittingAirbyteMessages(MockStreamWithState): + def __init__( + self, inputs_and_mocked_outputs: List[Tuple[Mapping[str, Any], Iterable[AirbyteMessage]]] = None, name: str = None, state=None + ): + super().__init__(inputs_and_mocked_outputs, name, state) + self._inputs_and_mocked_outputs = inputs_and_mocked_outputs + self._name = name + + @property + def name(self): + return self._name + + @property + def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: + return "pk" + + @property + def state(self) -> MutableMapping[str, Any]: + return {self.cursor_field: self._cursor_value} if self._cursor_value else {} + + @state.setter + def state(self, value: MutableMapping[str, Any]): + self._cursor_value = value.get(self.cursor_field, self.start_date) + + +def test_discover(mocker): + """Tests that the appropriate AirbyteCatalog is returned from the discover method""" + airbyte_stream1 = AirbyteStream( + name="1", + json_schema={}, + supported_sync_modes=[SyncMode.full_refresh, SyncMode.incremental], + default_cursor_field=["cursor"], + source_defined_cursor=True, + source_defined_primary_key=[["pk"]], + ) + airbyte_stream2 = AirbyteStream(name="2", json_schema={}, supported_sync_modes=[SyncMode.full_refresh]) + + stream1 = MockStream() + stream2 = MockStream() + mocker.patch.object(stream1, "as_airbyte_stream", return_value=airbyte_stream1) + mocker.patch.object(stream2, "as_airbyte_stream", return_value=airbyte_stream2) + + expected = AirbyteCatalog(streams=[airbyte_stream1, airbyte_stream2]) + src = MockSource(check_lambda=lambda: (True, None), streams=[stream1, stream2]) + + assert expected == src.discover(logger, {}) + + +def test_read_nonexistent_stream_raises_exception(mocker): + """Tests that attempting to sync a stream which the source does not return from the `streams` method raises an exception""" + s1 = MockStream(name="s1") + s2 = MockStream(name="this_stream_doesnt_exist_in_the_source") + + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + + src = MockSource(streams=[s1]) + catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(s2, SyncMode.full_refresh)]) + with pytest.raises(KeyError): + list(src.read(logger, {}, catalog)) + + +def test_read_nonexistent_stream_without_raises_exception(mocker): + """Tests that attempting to sync a stream which the source does not return from the `streams` method raises an exception""" + s1 = MockStream(name="s1") + s2 = MockStream(name="this_stream_doesnt_exist_in_the_source") + + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + + src = MockSource(streams=[s1], exception_on_missing_stream=False) + + catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(s2, SyncMode.full_refresh)]) + messages = list(src.read(logger, {}, catalog)) + + assert messages == [] + + +async def fake_read_records(*args, **kwargs): + for record in [{"a record": "a value"}, {"another record": "another value"}]: + yield record + + +def test_read_stream_emits_repository_message_before_record(mocker, message_repository): + stream = MockStream(name="my_stream") + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "read_records", fake_read_records) + message_repository.consume_queue.side_effect = [[message for message in [MESSAGE_FROM_REPOSITORY]], []] + + source = MockSource(streams=[stream], message_repository=message_repository) + + messages = list(source.read(logger, {}, ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]))) + + assert messages.count(MESSAGE_FROM_REPOSITORY) == 1 + record_messages = (message for message in messages if message.type == Type.RECORD) + assert all(messages.index(MESSAGE_FROM_REPOSITORY) < messages.index(record) for record in record_messages) + + +def test_read_stream_emits_repository_message_on_error(mocker, message_repository): + stream = MockStream(name="my_stream") + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "read_records", side_effect=RuntimeError("error")) + message_repository.consume_queue.return_value = [message for message in [MESSAGE_FROM_REPOSITORY]] + + source = MockSource(streams=[stream], message_repository=message_repository) + with pytest.raises(RuntimeError): + messages = list(source.read(logger, {}, ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]))) + assert MESSAGE_FROM_REPOSITORY in messages + + +async def fake_get_error_display_message(*args, **kwargs): + return "my message" + + +def test_read_stream_with_error_no_display_message(mocker): + stream = MockStream(name="my_stream") + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "read_records", side_effect=RuntimeError("oh no!")) + + source = MockSource(streams=[stream]) + catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) + + with pytest.raises(RuntimeError, match="oh no!"): + list(source.read(logger, {}, catalog)) + + +def test_read_stream_with_error_with_display_message(mocker): + stream = MockStream(name="my_stream") + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "read_records", side_effect=RuntimeError("oh no!")) + mocker.patch.object(stream, "get_error_display_message", fake_get_error_display_message) + + source = MockSource(streams=[stream]) + catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) + + with pytest.raises(AirbyteTracedException, match="oh no!") as exc: + list(source.read(logger, {}, catalog)) + assert exc.value.message == "my message" + + +GLOBAL_EMITTED_AT = 1 + + +def _as_record(stream: str, data: Dict[str, Any]) -> AirbyteMessage: + return AirbyteMessage( + type=Type.RECORD, + record=AirbyteRecordMessage(stream=stream, data=data, emitted_at=GLOBAL_EMITTED_AT), + ) + + +def _as_records(stream: str, data: List[Dict[str, Any]]) -> List[AirbyteMessage]: + return [_as_record(stream, datum) for datum in data] + + +def _as_stream_status(stream: str, status: AirbyteStreamStatus) -> AirbyteMessage: + trace_message = AirbyteTraceMessage( + emitted_at=datetime.datetime.now().timestamp() * 1000.0, + type=TraceType.STREAM_STATUS, + stream_status=AirbyteStreamStatusTraceMessage( + stream_descriptor=StreamDescriptor(name=stream), + status=status, + ), + ) + + return AirbyteMessage(type=MessageType.TRACE, trace=trace_message) + + +def _as_state(state_data: Dict[str, Any], stream_name: str = "", per_stream_state: Dict[str, Any] = None): + if per_stream_state: + return AirbyteMessage( + type=Type.STATE, + state=AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(name=stream_name), stream_state=AirbyteStateBlob.parse_obj(per_stream_state) + ), + data=state_data, + ), + ) + return AirbyteMessage(type=Type.STATE, state=AirbyteStateMessage(data=state_data)) + + +def _configured_stream(stream: AsyncStream, sync_mode: SyncMode): + return ConfiguredAirbyteStream( + stream=stream.as_airbyte_stream(), + sync_mode=sync_mode, + destination_sync_mode=DestinationSyncMode.overwrite, + ) + + +def _fix_emitted_at(messages: List[AirbyteMessage]) -> List[AirbyteMessage]: + for msg in messages: + if msg.type == Type.RECORD and msg.record: + msg.record.emitted_at = GLOBAL_EMITTED_AT + if msg.type == Type.TRACE and msg.trace: + msg.trace.emitted_at = GLOBAL_EMITTED_AT + return messages + + +def test_valid_full_refresh_read_no_slices(mocker): + """Tests that running a full refresh sync on streams which don't specify slices produces the expected AirbyteMessages""" + stream_output = [{"k1": "v1"}, {"k2": "v2"}] + s1 = MockStream([({"sync_mode": SyncMode.full_refresh}, stream_output)], name="s1") + s2 = MockStream([({"sync_mode": SyncMode.full_refresh}, stream_output)], name="s2") + + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + + src = MockSource(streams=[s1, s2]) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(s1, SyncMode.full_refresh), + _configured_stream(s2, SyncMode.full_refresh), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + *_as_records("s1", stream_output), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + *_as_records("s2", stream_output), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + messages = _fix_emitted_at(list(src.read(logger, {}, catalog))) + + assert expected == messages + + +def test_valid_full_refresh_read_with_slices(mocker): + """Tests that running a full refresh sync on streams which use slices produces the expected AirbyteMessages""" + # When attempting to sync a slice, just output that slice as a record + slices = [{"1": "1"}, {"2": "2"}] + s1 = MockStream( + [({"sync_mode": SyncMode.full_refresh, "stream_slice": s}, [s]) for s in slices], + name="s1", + ) + s2 = MockStream( + [({"sync_mode": SyncMode.full_refresh, "stream_slice": s}, [s]) for s in slices], + name="s2", + ) + + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) + + src = MockSource(streams=[s1, s2]) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(s1, SyncMode.full_refresh), + _configured_stream(s2, SyncMode.full_refresh), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + *_as_records("s1", slices), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + *_as_records("s2", slices), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + + messages = _fix_emitted_at(list(src.read(logger, {}, catalog))) + + assert expected == messages + + +@pytest.mark.parametrize( + "slices", + [ + [{"1": "1"}, {"2": "2"}], + [ + {"date": datetime.date(year=2023, month=1, day=1)}, + {"date": datetime.date(year=2023, month=1, day=1)}, + ] + ], +) +def test_read_full_refresh_with_slices_sends_slice_messages(mocker, slices): + """Given the logger is debug and a full refresh, AirbyteMessages are sent for slices""" + debug_logger = logging.getLogger("airbyte.debug") + debug_logger.setLevel(logging.DEBUG) + stream = MockStream( + [({"sync_mode": SyncMode.full_refresh, "stream_slice": s}, [s]) for s in slices], + name="s1", + ) + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) + + src = MockSource(streams=[stream]) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream, SyncMode.full_refresh), + ] + ) + + messages = src.read(debug_logger, {}, catalog) + + assert 2 == len(list(filter(lambda message: message.log and message.log.message.startswith("slice:"), messages))) + + +def test_read_incremental_with_slices_sends_slice_messages(mocker): + """Given the logger is debug and a incremental, AirbyteMessages are sent for slices""" + debug_logger = logging.getLogger("airbyte.debug") + debug_logger.setLevel(logging.DEBUG) + slices = [{"1": "1"}, {"2": "2"}] + stream = MockStream( + [({"sync_mode": SyncMode.incremental, "stream_slice": s, "stream_state": {}}, [s]) for s in slices], + name="s1", + ) + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + MockStream.supports_incremental = mocker.PropertyMock(return_value=True) + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) + + src = MockSource(streams=[stream]) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream, SyncMode.incremental), + ] + ) + + messages = src.read(debug_logger, {}, catalog) + + assert 2 == len(list(filter(lambda message: message.log and message.log.message.startswith("slice:"), messages))) + + +class TestIncrementalRead: + @pytest.mark.parametrize( + "use_legacy", + [ + pytest.param(True, id="test_incoming_stream_state_as_legacy_format"), + pytest.param(False, id="test_incoming_stream_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(True, id="test_source_emits_state_as_per_stream_format"), + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + def test_with_state_attribute(self, mocker, use_legacy, per_stream_enabled): + """Test correct state passing for the streams that have a state attribute""" + stream_output = [{"k1": "v1"}, {"k2": "v2"}] + old_state = {"cursor": "old_value"} + if use_legacy: + input_state = {"s1": old_state} + else: + input_state = [ + AirbyteStateMessage( + type=AirbyteStateType.STREAM, + stream=AirbyteStreamState( + stream_descriptor=StreamDescriptor(name="s1"), stream_state=AirbyteStateBlob.parse_obj(old_state) + ), + ), + ] + new_state_from_connector = {"cursor": "new_value"} + + stream_1 = MockStreamWithState( + [ + ( + {"sync_mode": SyncMode.incremental, "stream_state": old_state}, + stream_output, + ) + ], + name="s1", + ) + stream_2 = MockStreamWithState( + [({"sync_mode": SyncMode.incremental, "stream_state": {}}, stream_output)], + name="s2", + ) + mocker.patch.object(MockStreamWithState, "get_updated_state", return_value={}) + state_property = mocker.patch.object( + MockStreamWithState, + "state", + new_callable=mocker.PropertyMock, + return_value=new_state_from_connector, + ) + mocker.patch.object(MockStreamWithState, "get_json_schema", return_value={}) + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + _as_record("s1", stream_output[0]), + _as_record("s1", stream_output[1]), + _as_state({"s1": new_state_from_connector}, "s1", new_state_from_connector) + if per_stream_enabled + else _as_state({"s1": new_state_from_connector}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + _as_record("s2", stream_output[0]), + _as_record("s2", stream_output[1]), + _as_state({"s1": new_state_from_connector, "s2": new_state_from_connector}, "s2", new_state_from_connector) + if per_stream_enabled + else _as_state({"s1": new_state_from_connector, "s2": new_state_from_connector}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert messages == expected + assert state_property.mock_calls == [ + call(old_state), # set state for s1 + call(), # get state in the end of slice for s1 + call(), # get state in the end of slice for s2 + ] + + @pytest.mark.parametrize( + "use_legacy", + [ + pytest.param(True, id="test_incoming_stream_state_as_legacy_format"), + pytest.param(False, id="test_incoming_stream_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(True, id="test_source_emits_state_as_per_stream_format"), + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + def test_with_checkpoint_interval(self, mocker, use_legacy, per_stream_enabled): + """Tests that an incremental read which doesn't specify a checkpoint interval outputs a STATE message + after reading N records within a stream. + """ + if use_legacy: + input_state = defaultdict(dict) + else: + input_state = [] + stream_output = [{"k1": "v1"}, {"k2": "v2"}] + + stream_1 = MockStream( + [({"sync_mode": SyncMode.incremental, "stream_state": {}}, stream_output)], + name="s1", + ) + stream_2 = MockStream( + [({"sync_mode": SyncMode.incremental, "stream_state": {}}, stream_output)], + name="s2", + ) + state = {"cursor": "value"} + mocker.patch.object(MockStream, "get_updated_state", return_value=state) + mocker.patch.object(MockStream, "supports_incremental", return_value=True) + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + # Tell the source to output one state message per record + mocker.patch.object( + MockStream, + "state_checkpoint_interval", + new_callable=mocker.PropertyMock, + return_value=1, + ) + + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + _as_record("s1", stream_output[0]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_record("s1", stream_output[1]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + _as_record("s2", stream_output[0]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_record("s2", stream_output[1]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert expected == messages + + @pytest.mark.parametrize( + "use_legacy", + [ + pytest.param(True, id="test_incoming_stream_state_as_legacy_format"), + pytest.param(False, id="test_incoming_stream_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(True, id="test_source_emits_state_as_per_stream_format"), + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + def test_with_no_interval(self, mocker, use_legacy, per_stream_enabled): + """Tests that an incremental read which doesn't specify a checkpoint interval outputs + a STATE message only after fully reading the stream and does not output any STATE messages during syncing the stream. + """ + if use_legacy: + input_state = defaultdict(dict) + else: + input_state = [] + stream_output = [{"k1": "v1"}, {"k2": "v2"}] + + stream_1 = MockStream( + [({"sync_mode": SyncMode.incremental, "stream_state": {}}, stream_output)], + name="s1", + ) + stream_2 = MockStream( + [({"sync_mode": SyncMode.incremental, "stream_state": {}}, stream_output)], + name="s2", + ) + state = {"cursor": "value"} + mocker.patch.object(MockStream, "get_updated_state", return_value=state) + mocker.patch.object(MockStream, "supports_incremental", return_value=True) + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + *_as_records("s1", stream_output), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + *_as_records("s2", stream_output), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert expected == messages + + @pytest.mark.parametrize( + "use_legacy", + [ + pytest.param(True, id="test_incoming_stream_state_as_legacy_format"), + pytest.param(False, id="test_incoming_stream_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(True, id="test_source_emits_state_as_per_stream_format"), + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + def test_with_slices(self, mocker, use_legacy, per_stream_enabled): + """Tests that an incremental read which uses slices outputs each record in the slice followed by a STATE message, for each slice""" + if use_legacy: + input_state = defaultdict(dict) + else: + input_state = [] + slices = [{"1": "1"}, {"2": "2"}] + stream_output = [{"k1": "v1"}, {"k2": "v2"}, {"k3": "v3"}] + + stream_1 = MockStream( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s1", + ) + stream_2 = MockStream( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s2", + ) + + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + state = {"cursor": "value"} + mocker.patch.object(MockStream, "get_updated_state", return_value=state) + mocker.patch.object(MockStream, "supports_incremental", return_value=True) + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) + + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + # stream 1 slice 1 + *_as_records("s1", stream_output), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + # stream 1 slice 2 + *_as_records("s1", stream_output), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + # stream 2 slice 1 + *_as_records("s2", stream_output), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + # stream 2 slice 2 + *_as_records("s2", stream_output), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert expected == messages + + @pytest.mark.parametrize( + "use_legacy", + [ + pytest.param(True, id="test_incoming_stream_state_as_legacy_format"), + pytest.param(False, id="test_incoming_stream_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(True, id="test_source_emits_state_as_per_stream_format"), + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize("slices", [pytest.param([], id="test_slices_as_list"), pytest.param(iter([]), id="test_slices_as_iterator")]) + def test_no_slices(self, mocker, use_legacy, per_stream_enabled, slices): + """ + Tests that an incremental read returns at least one state messages even if no records were read: + 1. outputs a state message after reading the entire stream + """ + if use_legacy: + input_state = defaultdict(dict) + else: + input_state = [] + + stream_output = [{"k1": "v1"}, {"k2": "v2"}, {"k3": "v3"}] + state = {"cursor": "value"} + stream_1 = MockStreamWithState( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s1", + state=state, + ) + stream_2 = MockStreamWithState( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s2", + state=state, + ) + + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + mocker.patch.object(MockStreamWithState, "supports_incremental", return_value=True) + mocker.patch.object(MockStreamWithState, "get_json_schema", return_value={}) + mocker.patch.object(MockStreamWithState, "stream_slices", _fake_stream_slices) + mocker.patch.object( + MockStreamWithState, + "state_checkpoint_interval", + new_callable=mocker.PropertyMock, + return_value=2, + ) + + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert expected == messages + + @pytest.mark.parametrize( + "use_legacy", + [ + pytest.param(True, id="test_incoming_stream_state_as_legacy_format"), + pytest.param(False, id="test_incoming_stream_state_as_per_stream_format"), + ], + ) + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(True, id="test_source_emits_state_as_per_stream_format"), + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + def test_with_slices_and_interval(self, mocker, use_legacy, per_stream_enabled): + """ + Tests that an incremental read which uses slices and a checkpoint interval: + 1. outputs all records + 2. outputs a state message every N records (N=checkpoint_interval) + 3. outputs a state message after reading the entire slice + """ + if use_legacy: + input_state = defaultdict(dict) + else: + input_state = [] + slices = [{"1": "1"}, {"2": "2"}] + stream_output = [{"k1": "v1"}, {"k2": "v2"}, {"k3": "v3"}] + stream_1 = MockStream( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s1", + ) + stream_2 = MockStream( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s2", + ) + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + state = {"cursor": "value"} + mocker.patch.object(MockStream, "get_updated_state", return_value=state) + mocker.patch.object(MockStream, "supports_incremental", return_value=True) + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) + mocker.patch.object( + MockStream, + "state_checkpoint_interval", + new_callable=mocker.PropertyMock, + return_value=2, + ) + + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + # stream 1 slice 1 + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + _as_record("s1", stream_output[0]), + _as_record("s1", stream_output[1]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_record("s1", stream_output[2]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + # stream 1 slice 2 + _as_record("s1", stream_output[0]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_record("s1", stream_output[1]), + _as_record("s1", stream_output[2]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + # stream 2 slice 1 + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + _as_record("s2", stream_output[0]), + _as_record("s2", stream_output[1]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_record("s2", stream_output[2]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + # stream 2 slice 2 + _as_record("s2", stream_output[0]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_record("s2", stream_output[1]), + _as_record("s2", stream_output[2]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert messages == expected + + @pytest.mark.parametrize( + "per_stream_enabled", + [ + pytest.param(False, id="test_source_emits_state_as_per_stream_format"), + ], + ) + def test_emit_non_records(self, mocker, per_stream_enabled): + """ + Tests that an incremental read which uses slices and a checkpoint interval: + 1. outputs all records + 2. outputs a state message every N records (N=checkpoint_interval) + 3. outputs a state message after reading the entire slice + """ + + input_state = [] + slices = [{"1": "1"}, {"2": "2"}] + stream_output = [ + {"k1": "v1"}, + AirbyteLogMessage(level=Level.INFO, message="HELLO"), + {"k2": "v2"}, + {"k3": "v3"}, + ] + stream_1 = MockStreamEmittingAirbyteMessages( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s1", + state=copy.deepcopy(input_state), + ) + stream_2 = MockStreamEmittingAirbyteMessages( + [ + ( + { + "sync_mode": SyncMode.incremental, + "stream_slice": s, + "stream_state": mocker.ANY, + }, + stream_output, + ) + for s in slices + ], + name="s2", + state=copy.deepcopy(input_state), + ) + async def _fake_stream_slices(*args, **kwargs): + for _slice in slices: + yield _slice + + state = {"cursor": "value"} + mocker.patch.object(MockStream, "get_updated_state", return_value=state) + mocker.patch.object(MockStream, "supports_incremental", return_value=True) + mocker.patch.object(MockStream, "get_json_schema", return_value={}) + mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) + mocker.patch.object( + MockStream, + "state_checkpoint_interval", + new_callable=mocker.PropertyMock, + return_value=2, + ) + + src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + catalog = ConfiguredAirbyteCatalog( + streams=[ + _configured_stream(stream_1, SyncMode.incremental), + _configured_stream(stream_2, SyncMode.incremental), + ] + ) + + expected = _fix_emitted_at( + [ + _as_stream_status("s1", AirbyteStreamStatus.STARTED), + _as_stream_status("s1", AirbyteStreamStatus.RUNNING), + # stream 1 slice 1 + stream_data_to_airbyte_message("s1", stream_output[0]), + stream_data_to_airbyte_message("s1", stream_output[1]), + stream_data_to_airbyte_message("s1", stream_output[2]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + stream_data_to_airbyte_message("s1", stream_output[3]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + # stream 1 slice 2 + stream_data_to_airbyte_message("s1", stream_output[0]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + stream_data_to_airbyte_message("s1", stream_output[1]), + stream_data_to_airbyte_message("s1", stream_output[2]), + stream_data_to_airbyte_message("s1", stream_output[3]), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_state({"s1": state}, "s1", state) if per_stream_enabled else _as_state({"s1": state}), + _as_stream_status("s1", AirbyteStreamStatus.COMPLETE), + # stream 2 slice 1 + _as_stream_status("s2", AirbyteStreamStatus.STARTED), + _as_stream_status("s2", AirbyteStreamStatus.RUNNING), + stream_data_to_airbyte_message("s2", stream_output[0]), + stream_data_to_airbyte_message("s2", stream_output[1]), + stream_data_to_airbyte_message("s2", stream_output[2]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + stream_data_to_airbyte_message("s2", stream_output[3]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + # stream 2 slice 2 + stream_data_to_airbyte_message("s2", stream_output[0]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + stream_data_to_airbyte_message("s2", stream_output[1]), + stream_data_to_airbyte_message("s2", stream_output[2]), + stream_data_to_airbyte_message("s2", stream_output[3]), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_state({"s1": state, "s2": state}, "s2", state) if per_stream_enabled else _as_state({"s1": state, "s2": state}), + _as_stream_status("s2", AirbyteStreamStatus.COMPLETE), + ] + ) + + messages = _fix_emitted_at(list(src.read(logger, {}, catalog, state=input_state))) + + assert messages == expected + + +def test_checkpoint_state_from_stream_instance(): + teams_stream = MockStreamOverridesStateMethod() + managers_stream = StreamNoStateMethod() + state_manager = ConnectorStateManager({"teams": teams_stream, "managers": managers_stream}, []) + + # The stream_state passed to checkpoint_state() should be ignored since stream implements state function + teams_stream.state = {"updated_at": "2022-09-11"} + actual_message = teams_stream._checkpoint_state({"ignored": "state"}, state_manager, True) + assert actual_message == _as_state({"teams": {"updated_at": "2022-09-11"}}, "teams", {"updated_at": "2022-09-11"}) + + # The stream_state passed to checkpoint_state() should be used since the stream does not implement state function + actual_message = managers_stream._checkpoint_state({"updated": "expected_here"}, state_manager, True) + assert actual_message == _as_state( + {"teams": {"updated_at": "2022-09-11"}, "managers": {"updated": "expected_here"}}, "managers", {"updated": "expected_here"} + ) From 5840f9d54dde330e4099d4c5b4b5c26dd13d28a4 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Wed, 27 Dec 2023 22:27:41 -0500 Subject: [PATCH 40/74] WIP: test_http_async.py --- .../sources/streams/http/http_async.py | 21 +- .../sources/streams/http/test_http_async.py | 642 ++++++++++++++++++ .../source_salesforce/streams.py | 6 +- 3 files changed, 657 insertions(+), 12 deletions(-) create mode 100644 airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 8bbd2d7929bd..d9839d165d65 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -18,6 +18,7 @@ from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException @@ -39,14 +40,17 @@ class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): Basic building block for users building an Airbyte source for an async HTTP API. """ - def __init__(self, authenticator: HttpAuthenticator, api_budget: Optional[APIBudget] = None): + def __init__(self, authenticator: Optional[Union[HttpAuthenticator, NoAuth]] = NoAuth(), api_budget: Optional[APIBudget] = None): self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) self._session: aiohttp.ClientSession = None # TODO: HttpStream handles other authentication codepaths, which may need to be added later self._authenticator = authenticator @property - def authenticator(self) -> HttpAuthenticator: + def authenticator(self) -> Optional[Union[HttpAuthenticator, NoAuth]]: + # TODO: this behaves differently than http.py, which would return None if self._authenticator is an HttpAuthenticator. + # But, it looks like this property is only used here in http_async.py and Salesforce's streams.py. + # It doesn't appear to be causing any problems with Salesforce. return self._authenticator @property @@ -169,7 +173,7 @@ def _create_aiohttp_client_request( data: Optional[Union[str, Mapping[str, Any]]] = None, ) -> aiohttp.ClientRequest: str_url = self._join_url(self.url_base, path) - str_url = "http://localhost:8000" + str_url = "http://localhost:8000" # TODO url = URL(str_url) if self.must_deduplicate_query_params(): query_params = self.deduplicate_query_params(str_url, params) @@ -216,8 +220,6 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st request.method, request.url, headers=request.headers, auth=request.auth, - chunked=request.chunked, - compress=request.compress, **request_kwargs, ) @@ -350,10 +352,11 @@ async def read_records( stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[StreamData]: - async for record in self._read_pages( - lambda req, res, state, _slice: self.parse_response(res, stream_slice=_slice, stream_state=state), stream_slice, - stream_state - ): + async def _records_generator_fn(req, res, state, _slice): + async for record in self.parse_response(res, stream_slice=_slice, stream_state=state): + yield record + + async for record in self._read_pages(_records_generator_fn, stream_slice, stream_state): yield record async def _read_pages( diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py new file mode 100644 index 000000000000..beb8b1885ea1 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -0,0 +1,642 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + + +import asyncio +import json +from http import HTTPStatus +from typing import Any, Iterable, Mapping, Optional +from unittest.mock import ANY, MagicMock, patch + +import aiohttp +import pytest +import requests +from aioresponses import aioresponses +from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.streams.http.auth import NoAuth +from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator as HttpTokenAuthenticator +from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream +from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException +from airbyte_cdk.sources.streams.http.requests_native_auth import TokenAuthenticator + + +class StubBasicReadHttpStream(AsyncHttpStream): + url_base = "https://test_base_url.com" + primary_key = "" + + def __init__(self, deduplicate_query_params: bool = False, **kwargs): + super().__init__(**kwargs) + self.resp_counter = 1 + self._deduplicate_query_params = deduplicate_query_params + + async def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + return None + + def path(self, **kwargs) -> str: + return "" + + async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + stubResp = {"data": self.resp_counter} + self.resp_counter += 1 + yield stubResp + + def must_deduplicate_query_params(self) -> bool: + return self._deduplicate_query_params + + +def test_default_authenticator(): + stream = StubBasicReadHttpStream() + assert isinstance(stream.authenticator, NoAuth) + + +def test_http_token_authenticator(): + stream = StubBasicReadHttpStream(authenticator=HttpTokenAuthenticator("test-token")) + assert isinstance(stream.authenticator, HttpTokenAuthenticator) + + +def test_request_kwargs_used(mocker): + loop = asyncio.get_event_loop() + stream = StubBasicReadHttpStream() + loop.run_until_complete(stream.ensure_session()) + + request_kwargs = {"chunked": True, "compress": True} + mocker.patch.object(stream, "request_kwargs", return_value=request_kwargs) + + with aioresponses() as m: + m.get(stream.url_base, status=200) + loop.run_until_complete(read_records(stream)) + + m.assert_any_call(stream.url_base, "GET", **request_kwargs) + m.assert_called_once() + + +async def read_records(stream, sync_mode=SyncMode.full_refresh): + records = [] + async for record in stream.read_records(sync_mode=sync_mode): + records.append(record) + return records + + +def test_stub_basic_read_http_stream_read_records(mocker): + loop = asyncio.get_event_loop() + stream = StubBasicReadHttpStream() + blank_response = {} # Send a blank response is fine as we ignore the response in `parse_response anyway. + mocker.patch.object(StubBasicReadHttpStream, "_send_request", return_value=blank_response) + + records = loop.run_until_complete(read_records(stream)) + + assert [{"data": 1}] == records + + +class StubNextPageTokenHttpStream(StubBasicReadHttpStream): + current_page = 0 + + def __init__(self, pages: int = 5): + super().__init__() + self._pages = pages + + async def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + while self.current_page < self._pages: + page_token = {"page": self.current_page} + self.current_page += 1 + return page_token + return None + + +def test_next_page_token_is_input_to_other_methods(mocker): + """Validates that the return value from next_page_token is passed into other methods that need it like request_params, headers, body, etc..""" + pages = 5 + stream = StubNextPageTokenHttpStream(pages=pages) + blank_response = {} # Send a blank response is fine as we ignore the response in `parse_response anyway. + mocker.patch.object(StubNextPageTokenHttpStream, "_send_request", return_value=blank_response) + + methods = ["request_params", "request_headers", "request_body_json"] + for method in methods: + # Wrap all methods we're interested in testing with mocked objects so we can later spy on their input args and verify they were what we expect + mocker.patch.object(stream, method, wraps=getattr(stream, method)) + + loop = asyncio.get_event_loop() + records = loop.run_until_complete(read_records(stream)) + + # Since we have 5 pages, we expect 5 tokens which are {"page":1}, {"page":2}, etc... + expected_next_page_tokens = [{"page": i} for i in range(pages)] + for method in methods: + # First assert that they were called with no next_page_token. This is the first call in the pagination loop. + getattr(stream, method).assert_any_call(next_page_token=None, stream_slice=None, stream_state={}) + for token in expected_next_page_tokens: + # Then verify that each method + getattr(stream, method).assert_any_call(next_page_token=token, stream_slice=None, stream_state={}) + + expected = [{"data": 1}, {"data": 2}, {"data": 3}, {"data": 4}, {"data": 5}, {"data": 6}] + + assert expected == records + + +class StubBadUrlHttpStream(StubBasicReadHttpStream): + url_base = "bad_url" + + +def test_stub_bad_url_http_stream_read_records(): + stream = StubBadUrlHttpStream() + loop = asyncio.get_event_loop() + with pytest.raises(aiohttp.client_exceptions.InvalidURL): + loop.run_until_complete(read_records(stream)) + + +class StubCustomBackoffHttpStream(StubBasicReadHttpStream): + def backoff_time(self, response: requests.Response) -> Optional[float]: + return 0.5 + + +def test_stub_custom_backoff_http_stream(mocker): + mocker.patch("time.sleep", lambda x: None) + stream = StubCustomBackoffHttpStream() + req = requests.Response() + req.status_code = 429 + + send_mock = mocker.patch.object(requests.Session, "send", return_value=req) + + with pytest.raises(UserDefinedBackoffException): + list(stream.read_records(SyncMode.full_refresh)) + assert send_mock.call_count == stream.max_retries + 1 + + # TODO(davin): Figure out how to assert calls. + + +@pytest.mark.parametrize("retries", [-20, -1, 0, 1, 2, 10]) +def test_stub_custom_backoff_http_stream_retries(mocker, retries): + mocker.patch("time.sleep", lambda x: None) + + class StubCustomBackoffHttpStreamRetries(StubCustomBackoffHttpStream): + @property + def max_retries(self): + return retries + + stream = StubCustomBackoffHttpStreamRetries() + req = requests.Response() + req.status_code = HTTPStatus.TOO_MANY_REQUESTS + send_mock = mocker.patch.object(requests.Session, "send", return_value=req) + + with pytest.raises(UserDefinedBackoffException, match="Request URL: https://test_base_url.com/, Response Code: 429") as excinfo: + list(stream.read_records(SyncMode.full_refresh)) + assert isinstance(excinfo.value.request, requests.PreparedRequest) + assert isinstance(excinfo.value.response, requests.Response) + if retries <= 0: + assert send_mock.call_count == 1 + else: + assert send_mock.call_count == stream.max_retries + 1 + + +def test_stub_custom_backoff_http_stream_endless_retries(mocker): + mocker.patch("time.sleep", lambda x: None) + + class StubCustomBackoffHttpStreamRetries(StubCustomBackoffHttpStream): + @property + def max_retries(self): + return None + + infinite_number = 20 + + stream = StubCustomBackoffHttpStreamRetries() + req = requests.Response() + req.status_code = HTTPStatus.TOO_MANY_REQUESTS + send_mock = mocker.patch.object(requests.Session, "send", side_effect=[req] * infinite_number) + + # Expecting mock object to raise a RuntimeError when the end of side_effect list parameter reached. + with pytest.raises(RuntimeError): + list(stream.read_records(SyncMode.full_refresh)) + assert send_mock.call_count == infinite_number + 1 + + +@pytest.mark.parametrize("http_code", [400, 401, 403]) +def test_4xx_error_codes_http_stream(mocker, http_code): + stream = StubCustomBackoffHttpStream() + req = requests.Response() + req.status_code = http_code + mocker.patch.object(requests.Session, "send", return_value=req) + + with pytest.raises(requests.exceptions.HTTPError): + list(stream.read_records(SyncMode.full_refresh)) + + +class AutoFailFalseHttpStream(StubBasicReadHttpStream): + raise_on_http_errors = False + max_retries = 3 + retry_factor = 0.01 + + +def test_raise_on_http_errors_off_429(mocker): + stream = AutoFailFalseHttpStream() + req = requests.Response() + req.status_code = 429 + + mocker.patch.object(requests.Session, "send", return_value=req) + with pytest.raises(DefaultBackoffException, match="Request URL: https://test_base_url.com/, Response Code: 429"): + list(stream.read_records(SyncMode.full_refresh)) + + +@pytest.mark.parametrize("status_code", [500, 501, 503, 504]) +def test_raise_on_http_errors_off_5xx(mocker, status_code): + stream = AutoFailFalseHttpStream() + req = requests.Response() + req.status_code = status_code + + send_mock = mocker.patch.object(requests.Session, "send", return_value=req) + with pytest.raises(DefaultBackoffException): + list(stream.read_records(SyncMode.full_refresh)) + assert send_mock.call_count == stream.max_retries + 1 + + +@pytest.mark.parametrize("status_code", [400, 401, 402, 403, 416]) +def test_raise_on_http_errors_off_non_retryable_4xx(mocker, status_code): + stream = AutoFailFalseHttpStream() + req = requests.PreparedRequest() + res = requests.Response() + res.status_code = status_code + + mocker.patch.object(requests.Session, "send", return_value=res) + response = stream._send_request(req, {}) + assert response.status_code == status_code + + +@pytest.mark.parametrize( + "error", + ( + requests.exceptions.ConnectTimeout, + requests.exceptions.ConnectionError, + requests.exceptions.ChunkedEncodingError, + requests.exceptions.ReadTimeout, + ), +) +def test_raise_on_http_errors(mocker, error): + stream = AutoFailFalseHttpStream() + send_mock = mocker.patch.object(requests.Session, "send", side_effect=error()) + + with pytest.raises(error): + list(stream.read_records(SyncMode.full_refresh)) + assert send_mock.call_count == stream.max_retries + 1 + + +class PostHttpStream(StubBasicReadHttpStream): + http_method = "POST" + + async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + """Returns response data as is""" + yield response.json() + + +class TestRequestBody: + """Suite of different tests for request bodies""" + + json_body = {"key": "value"} + data_body = "key:value" + form_body = {"key1": "value1", "key2": 1234} + urlencoded_form_body = "key1=value1&key2=1234" + + def request2response(self, request, context): + return json.dumps({"body": request.text, "content_type": request.headers.get("Content-Type")}) + + def test_json_body(self, mocker, requests_mock): + + stream = PostHttpStream() + mocker.patch.object(stream, "request_body_json", return_value=self.json_body) + + requests_mock.register_uri("POST", stream.url_base, text=self.request2response) + response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + + assert response["content_type"] == "application/json" + assert json.loads(response["body"]) == self.json_body + + def test_text_body(self, mocker, requests_mock): + + stream = PostHttpStream() + mocker.patch.object(stream, "request_body_data", return_value=self.data_body) + + requests_mock.register_uri("POST", stream.url_base, text=self.request2response) + response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + + assert response["content_type"] is None + assert response["body"] == self.data_body + + def test_form_body(self, mocker, requests_mock): + + stream = PostHttpStream() + mocker.patch.object(stream, "request_body_data", return_value=self.form_body) + + requests_mock.register_uri("POST", stream.url_base, text=self.request2response) + response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + + assert response["content_type"] == "application/x-www-form-urlencoded" + assert response["body"] == self.urlencoded_form_body + + def test_text_json_body(self, mocker, requests_mock): + """checks a exception if both functions were overridden""" + stream = PostHttpStream() + mocker.patch.object(stream, "request_body_data", return_value=self.data_body) + mocker.patch.object(stream, "request_body_json", return_value=self.json_body) + requests_mock.register_uri("POST", stream.url_base, text=self.request2response) + with pytest.raises(RequestBodyException): + list(stream.read_records(sync_mode=SyncMode.full_refresh)) + + def test_body_for_all_methods(self, mocker, requests_mock): + """Stream must send a body for GET/POST/PATCH/PUT methods only""" + stream = PostHttpStream() + methods = { + "POST": True, + "PUT": True, + "PATCH": True, + "GET": True, + "DELETE": False, + "OPTIONS": False, + } + for method, with_body in methods.items(): + stream.http_method = method + mocker.patch.object(stream, "request_body_data", return_value=self.data_body) + requests_mock.register_uri(method, stream.url_base, text=self.request2response) + response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + if with_body: + assert response["body"] == self.data_body + else: + assert response["body"] is None + + +class CacheHttpStream(StubBasicReadHttpStream): + use_cache = True + + +class CacheHttpSubStream(AsyncHttpSubStream): + url_base = "https://example.com" + primary_key = "" + + def __init__(self, parent): + super().__init__(parent=parent) + + async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + yield None + + def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + return None + + def path(self, **kwargs) -> str: + return "" + + +def test_caching_filename(): + stream = CacheHttpStream() + assert stream.cache_filename == f"{stream.name}.sqlite" + + +def test_caching_sessions_are_different(): + stream_1 = CacheHttpStream() + stream_2 = CacheHttpStream() + + assert stream_1._session != stream_2._session + assert stream_1.cache_filename == stream_2.cache_filename + + +# def test_cached_streams_wortk_when_request_path_is_not_set(mocker, requests_mock): +# This test verifies that HttpStreams with a cached session work even if the path is not set +# For instance, when running in a unit test +# stream = CacheHttpStream() +# with mocker.patch.object(stream._session, "send", wraps=stream._session.send): +# requests_mock.register_uri("GET", stream.url_base) +# records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) +# assert records == [{"data": 1}] +# "" + + +def test_parent_attribute_exist(): + parent_stream = CacheHttpStream() + child_stream = CacheHttpSubStream(parent=parent_stream) + + assert child_stream.parent == parent_stream + + +def test_that_response_was_cached(mocker, requests_mock): + requests_mock.register_uri("GET", "https://google.com/", text="text") + stream = CacheHttpStream() + stream.clear_cache() + mocker.patch.object(stream, "url_base", "https://google.com/") + records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) + + assert requests_mock.called + + requests_mock.reset_mock() + new_records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) + + assert len(records) == len(new_records) + assert not requests_mock.called + + +class CacheHttpStreamWithSlices(CacheHttpStream): + paths = ["", "search"] + + def path(self, stream_slice: Mapping[str, Any] = None, **kwargs) -> str: + return f'{stream_slice["path"]}' if stream_slice else "" + + def stream_slices(self, **kwargs) -> Iterable[Optional[Mapping[str, Any]]]: + for path in self.paths: + yield {"path": path} + + async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + yield {"value": len(response.text)} + + +@patch("airbyte_cdk.sources.streams.core.logging", MagicMock()) +def test_using_cache(mocker, requests_mock): + requests_mock.register_uri("GET", "https://google.com/", text="text") + requests_mock.register_uri("GET", "https://google.com/search", text="text") + + parent_stream = CacheHttpStreamWithSlices() + mocker.patch.object(parent_stream, "url_base", "https://google.com/") + parent_stream.clear_cache() + + assert requests_mock.call_count == 0 + assert len(parent_stream._session.cache.responses) == 0 + + for _slice in parent_stream.stream_slices(): + list(parent_stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=_slice)) + + assert requests_mock.call_count == 2 + assert len(parent_stream._session.cache.responses) == 2 + + child_stream = CacheHttpSubStream(parent=parent_stream) + + for _slice in child_stream.stream_slices(sync_mode=SyncMode.full_refresh): + pass + + assert requests_mock.call_count == 2 + assert len(parent_stream._session.cache.responses) == 2 + assert parent_stream._session.cache.contains(url="https://google.com/") + assert parent_stream._session.cache.contains(url="https://google.com/search") + + +class AutoFailTrueHttpStream(StubBasicReadHttpStream): + raise_on_http_errors = True + + +@pytest.mark.parametrize("status_code", range(400, 600)) +def test_send_raise_on_http_errors_logs(mocker, status_code): + mocker.patch.object(AutoFailTrueHttpStream, "logger") + mocker.patch.object(AutoFailTrueHttpStream, "should_retry", mocker.Mock(return_value=False)) + stream = AutoFailTrueHttpStream() + req = requests.PreparedRequest() + res = requests.Response() + res.status_code = status_code + mocker.patch.object(requests.Session, "send", return_value=res) + with pytest.raises(requests.exceptions.HTTPError): + response = stream._send_request(req, {}) + stream.logger.error.assert_called_with(response.text) + assert response.status_code == status_code + + +@pytest.mark.parametrize( + "api_response, expected_message", + [ + ({"error": "something broke"}, "something broke"), + ({"error": {"message": "something broke"}}, "something broke"), + ({"error": "err-001", "message": "something broke"}, "something broke"), + ({"failure": {"message": "something broke"}}, "something broke"), + ({"error": {"errors": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}}, "one, two, three"), + ({"errors": ["one", "two", "three"]}, "one, two, three"), + ({"messages": ["one", "two", "three"]}, "one, two, three"), + ({"errors": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, "one, two, three"), + ({"error": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, "one, two, three"), + ({"errors": [{"error": "one"}, {"error": "two"}, {"error": "three"}]}, "one, two, three"), + ({"failures": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, "one, two, three"), + (["one", "two", "three"], "one, two, three"), + ([{"error": "one"}, {"error": "two"}, {"error": "three"}], "one, two, three"), + ({"error": True}, None), + ({"something_else": "hi"}, None), + ({}, None), + ], +) +def test_default_parse_response_error_message(api_response: dict, expected_message: Optional[str]): + stream = StubBasicReadHttpStream() + response = MagicMock() + response.json.return_value = api_response + + message = stream.parse_response_error_message(response) + assert message == expected_message + + +def test_default_parse_response_error_message_not_json(requests_mock): + stream = StubBasicReadHttpStream() + requests_mock.register_uri("GET", "mock://test.com/not_json", text="this is not json") + response = requests.get("mock://test.com/not_json") + + message = stream.parse_response_error_message(response) + assert message is None + + +def test_default_get_error_display_message_handles_http_error(mocker): + stream = StubBasicReadHttpStream() + mocker.patch.object(stream, "parse_response_error_message", return_value="my custom message") + + non_http_err_msg = stream.get_error_display_message(RuntimeError("not me")) + assert non_http_err_msg is None + + response = requests.Response() + http_exception = requests.HTTPError(response=response) + http_err_msg = stream.get_error_display_message(http_exception) + assert http_err_msg == "my custom message" + + +@pytest.mark.parametrize( + "test_name, base_url, path, expected_full_url", + [ + ("test_no_slashes", "https://airbyte.io", "my_endpoint", "https://airbyte.io/my_endpoint"), + ("test_trailing_slash_on_base_url", "https://airbyte.io/", "my_endpoint", "https://airbyte.io/my_endpoint"), + ( + "test_trailing_slash_on_base_url_and_leading_slash_on_path", + "https://airbyte.io/", + "/my_endpoint", + "https://airbyte.io/my_endpoint", + ), + ("test_leading_slash_on_path", "https://airbyte.io", "/my_endpoint", "https://airbyte.io/my_endpoint"), + ("test_trailing_slash_on_path", "https://airbyte.io", "/my_endpoint/", "https://airbyte.io/my_endpoint/"), + ("test_nested_path_no_leading_slash", "https://airbyte.io", "v1/my_endpoint", "https://airbyte.io/v1/my_endpoint"), + ("test_nested_path_with_leading_slash", "https://airbyte.io", "/v1/my_endpoint", "https://airbyte.io/v1/my_endpoint"), + ], +) +def test_join_url(test_name, base_url, path, expected_full_url): + actual_url = AsyncHttpStream._join_url(base_url, path) + assert actual_url == expected_full_url + + +@pytest.mark.parametrize( + "deduplicate_query_params, path, params, expected_url", + [ + pytest.param( + True, "v1/endpoint?param1=value1", {}, "https://test_base_url.com/v1/endpoint?param1=value1", id="test_params_only_in_path" + ), + pytest.param( + True, "v1/endpoint", {"param1": "value1"}, "https://test_base_url.com/v1/endpoint?param1=value1", id="test_params_only_in_path" + ), + pytest.param(True, "v1/endpoint", None, "https://test_base_url.com/v1/endpoint", id="test_params_is_none_and_no_params_in_path"), + pytest.param( + True, + "v1/endpoint?param1=value1", + None, + "https://test_base_url.com/v1/endpoint?param1=value1", + id="test_params_is_none_and_no_params_in_path", + ), + pytest.param( + True, + "v1/endpoint?param1=value1", + {"param2": "value2"}, + "https://test_base_url.com/v1/endpoint?param1=value1¶m2=value2", + id="test_no_duplicate_params", + ), + pytest.param( + True, + "v1/endpoint?param1=value1", + {"param1": "value1"}, + "https://test_base_url.com/v1/endpoint?param1=value1", + id="test_duplicate_params_same_value", + ), + pytest.param( + True, + "v1/endpoint?param1=1", + {"param1": 1}, + "https://test_base_url.com/v1/endpoint?param1=1", + id="test_duplicate_params_same_value_not_string", + ), + pytest.param( + True, + "v1/endpoint?param1=value1", + {"param1": "value2"}, + "https://test_base_url.com/v1/endpoint?param1=value1¶m1=value2", + id="test_duplicate_params_different_value", + ), + pytest.param( + False, + "v1/endpoint?param1=value1", + {"param1": "value2"}, + "https://test_base_url.com/v1/endpoint?param1=value1¶m1=value2", + id="test_same_params_different_value_no_deduplication", + ), + pytest.param( + False, + "v1/endpoint?param1=value1", + {"param1": "value1"}, + "https://test_base_url.com/v1/endpoint?param1=value1¶m1=value1", + id="test_same_params_same_value_no_deduplication", + ), + ], +) +def test_duplicate_request_params_are_deduped(deduplicate_query_params, path, params, expected_url): + stream = StubBasicReadHttpStream(deduplicate_query_params) + + if expected_url is None: + with pytest.raises(ValueError): + stream._create_prepared_request(path=path, params=params) + else: + prepared_request = stream._create_prepared_request(path=path, params=params) + assert prepared_request.url == expected_url + + +def test_connection_pool(): + stream = StubBasicReadHttpStream(authenticator=HttpTokenAuthenticator("test-token")) + assert stream._session.adapters["https://"]._pool_connections == 20 diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 4cba6e2f79b7..800f60f3aaf5 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -99,7 +99,8 @@ def too_many_properties(self): return properties_length > self.max_properties_length async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> List[Mapping]: - return (await response.json())["records"] + for record in (await response.json())["records"]: + yield record def get_json_schema(self) -> Mapping[str, Any]: if not self.schema: @@ -257,12 +258,11 @@ async def f(): # When this is the first time we're getting a chunk's records, we set this to False to be used when deciding the next chunk if property_chunk.first_time: property_chunk.first_time = False - chunk_page_records = await records_generator_fn(request, response, stream_state, stream_slice) if not self.too_many_properties: # this is the case when a stream has no primary key # (it is allowed when properties length does not exceed the maximum value) # so there would be a single chunk, therefore we may and should yield records immediately - for record in chunk_page_records: + async for record in records_generator_fn(request, response, stream_state, stream_slice): property_chunk.record_counter += 1 yield record continue From 298bff14012da39e792d34f9cbcee3e348f73611 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 28 Dec 2023 12:16:22 -0500 Subject: [PATCH 41/74] Fix backoff handling --- .../sources/streams/http/http_async.py | 2 +- .../streams/http/rate_limiting_async.py | 2 +- .../sources/streams/http/test_http_async.py | 208 +++++++++++------- 3 files changed, 135 insertions(+), 77 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index d9839d165d65..c6fe000bc811 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -26,7 +26,7 @@ from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH from .auth.core import HttpAuthenticator -from .rate_limiting import default_backoff_handler, user_defined_backoff_handler +from .rate_limiting_async import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py index 7fea5a9cedc8..0567f0115b9a 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py @@ -10,7 +10,7 @@ import aiohttp import backoff -from .exceptions import DefaultBackoffException, UserDefinedBackoffException +from .exceptions_async import DefaultBackoffException, UserDefinedBackoffException TRANSIENT_EXCEPTIONS = ( DefaultBackoffException, diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index beb8b1885ea1..44fc7b80b656 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -8,6 +8,7 @@ from http import HTTPStatus from typing import Any, Iterable, Mapping, Optional from unittest.mock import ANY, MagicMock, patch +from yarl import URL import aiohttp import pytest @@ -152,16 +153,21 @@ def backoff_time(self, response: requests.Response) -> Optional[float]: def test_stub_custom_backoff_http_stream(mocker): mocker.patch("time.sleep", lambda x: None) stream = StubCustomBackoffHttpStream() - req = requests.Response() - req.status_code = 429 + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + call_counter = 0 - send_mock = mocker.patch.object(requests.Session, "send", return_value=req) + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + + with aioresponses() as m: + m.get(stream.url_base, status=429, repeat=True, callback=request_callback) - with pytest.raises(UserDefinedBackoffException): - list(stream.read_records(SyncMode.full_refresh)) - assert send_mock.call_count == stream.max_retries + 1 + with pytest.raises(UserDefinedBackoffException): + loop.run_until_complete(read_records(stream)) - # TODO(davin): Figure out how to assert calls. + assert call_counter == stream.max_retries + 1 @pytest.mark.parametrize("retries", [-20, -1, 0, 1, 2, 10]) @@ -173,19 +179,27 @@ class StubCustomBackoffHttpStreamRetries(StubCustomBackoffHttpStream): def max_retries(self): return retries + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + stream = StubCustomBackoffHttpStreamRetries() - req = requests.Response() - req.status_code = HTTPStatus.TOO_MANY_REQUESTS - send_mock = mocker.patch.object(requests.Session, "send", return_value=req) - - with pytest.raises(UserDefinedBackoffException, match="Request URL: https://test_base_url.com/, Response Code: 429") as excinfo: - list(stream.read_records(SyncMode.full_refresh)) - assert isinstance(excinfo.value.request, requests.PreparedRequest) - assert isinstance(excinfo.value.response, requests.Response) - if retries <= 0: - assert send_mock.call_count == 1 - else: - assert send_mock.call_count == stream.max_retries + 1 + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + call_counter = 0 + + with aioresponses() as m: + m.get(stream.url_base, status=429, repeat=True, callback=request_callback) + + with pytest.raises(UserDefinedBackoffException) as excinfo: + loop.run_until_complete(read_records(stream)) + assert isinstance(excinfo.value.request, aiohttp.ClientRequest) + assert isinstance(excinfo.value.response, aiohttp.ClientResponse) + + if retries <= 0: + m.assert_called_once() + else: + assert call_counter == stream.max_retries + 1 def test_stub_custom_backoff_http_stream_endless_retries(mocker): @@ -196,28 +210,41 @@ class StubCustomBackoffHttpStreamRetries(StubCustomBackoffHttpStream): def max_retries(self): return None + stream = StubCustomBackoffHttpStreamRetries() + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) infinite_number = 20 + call_counter = 0 - stream = StubCustomBackoffHttpStreamRetries() - req = requests.Response() - req.status_code = HTTPStatus.TOO_MANY_REQUESTS - send_mock = mocker.patch.object(requests.Session, "send", side_effect=[req] * infinite_number) + with aioresponses() as m: + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + if call_counter > infinite_number: + # Simulate a different response or a break in the pattern + # to stop the infinite retries + raise RuntimeError("End of retries") - # Expecting mock object to raise a RuntimeError when the end of side_effect list parameter reached. - with pytest.raises(RuntimeError): - list(stream.read_records(SyncMode.full_refresh)) - assert send_mock.call_count == infinite_number + 1 + m.get(stream.url_base, status=HTTPStatus.TOO_MANY_REQUESTS, repeat=True, callback=request_callback) + + # Expecting mock object to raise a RuntimeError when the end of side_effect list parameter reached. + with pytest.raises(RuntimeError): + loop.run_until_complete(read_records(stream)) + + assert call_counter == infinite_number + 1 @pytest.mark.parametrize("http_code", [400, 401, 403]) -def test_4xx_error_codes_http_stream(mocker, http_code): +def test_4xx_error_codes_http_stream(http_code): stream = StubCustomBackoffHttpStream() - req = requests.Response() - req.status_code = http_code - mocker.patch.object(requests.Session, "send", return_value=req) + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) - with pytest.raises(requests.exceptions.HTTPError): - list(stream.read_records(SyncMode.full_refresh)) + with aioresponses() as m: + m.get(stream.url_base, status=http_code, repeat=True) + + with pytest.raises(aiohttp.ClientResponseError): + loop.run_until_complete(read_records(stream)) class AutoFailFalseHttpStream(StubBasicReadHttpStream): @@ -226,56 +253,74 @@ class AutoFailFalseHttpStream(StubBasicReadHttpStream): retry_factor = 0.01 -def test_raise_on_http_errors_off_429(mocker): +def test_raise_on_http_errors_off_429(): stream = AutoFailFalseHttpStream() - req = requests.Response() - req.status_code = 429 + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) - mocker.patch.object(requests.Session, "send", return_value=req) - with pytest.raises(DefaultBackoffException, match="Request URL: https://test_base_url.com/, Response Code: 429"): - list(stream.read_records(SyncMode.full_refresh)) + with aioresponses() as m: + m.get(stream.url_base, status=429, repeat=True) + with pytest.raises(DefaultBackoffException): + loop.run_until_complete(read_records(stream)) @pytest.mark.parametrize("status_code", [500, 501, 503, 504]) -def test_raise_on_http_errors_off_5xx(mocker, status_code): +def test_raise_on_http_errors_off_5xx(status_code): stream = AutoFailFalseHttpStream() - req = requests.Response() - req.status_code = status_code + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + call_counter = 0 - send_mock = mocker.patch.object(requests.Session, "send", return_value=req) - with pytest.raises(DefaultBackoffException): - list(stream.read_records(SyncMode.full_refresh)) - assert send_mock.call_count == stream.max_retries + 1 + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + + with aioresponses() as m: + m.get(stream.url_base, status=status_code, repeat=True, callback=request_callback) + with pytest.raises(DefaultBackoffException): + loop.run_until_complete(read_records(stream)) + + assert call_counter == stream.max_retries + 1 @pytest.mark.parametrize("status_code", [400, 401, 402, 403, 416]) -def test_raise_on_http_errors_off_non_retryable_4xx(mocker, status_code): +def test_raise_on_http_errors_off_non_retryable_4xx(status_code): stream = AutoFailFalseHttpStream() - req = requests.PreparedRequest() - res = requests.Response() - res.status_code = status_code + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) - mocker.patch.object(requests.Session, "send", return_value=res) - response = stream._send_request(req, {}) - assert response.status_code == status_code + with aioresponses() as m: + m.get(stream.url_base, status=status_code, repeat=True) + response = loop.run_until_complete(stream._send_request(aiohttp.ClientRequest("GET", URL(stream.url_base)), {})) + + assert response.status == status_code @pytest.mark.parametrize( "error", ( - requests.exceptions.ConnectTimeout, - requests.exceptions.ConnectionError, - requests.exceptions.ChunkedEncodingError, - requests.exceptions.ReadTimeout, + aiohttp.ServerDisconnectedError, + aiohttp.ServerConnectionError, + aiohttp.ServerTimeoutError, ), ) -def test_raise_on_http_errors(mocker, error): +def test_raise_on_http_errors(error): stream = AutoFailFalseHttpStream() - send_mock = mocker.patch.object(requests.Session, "send", side_effect=error()) + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + call_counter = 0 + + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + + with aioresponses() as m: + m.get(stream.url_base, repeat=True, callback=request_callback, exception=error()) + + with pytest.raises(error): + loop.run_until_complete(read_records(stream)) - with pytest.raises(error): - list(stream.read_records(SyncMode.full_refresh)) - assert send_mock.call_count == stream.max_retries + 1 + assert call_counter == stream.max_retries + 1 class PostHttpStream(StubBasicReadHttpStream): @@ -294,30 +339,43 @@ class TestRequestBody: form_body = {"key1": "value1", "key2": 1234} urlencoded_form_body = "key1=value1&key2=1234" - def request2response(self, request, context): - return json.dumps({"body": request.text, "content_type": request.headers.get("Content-Type")}) - - def test_json_body(self, mocker, requests_mock): + def request2response(self, **kwargs): + """Callback function to handle request and return mock response.""" + body = kwargs.get("data") + headers = kwargs.get("headers", {}) + return { + "body": json.dumps(body) if isinstance(body, dict) else body, + "content_type": headers.get("Content-Type") + } + def test_json_body(self, mocker): stream = PostHttpStream() mocker.patch.object(stream, "request_body_json", return_value=self.json_body) + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) - requests_mock.register_uri("POST", stream.url_base, text=self.request2response) - response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + with aioresponses() as m: + m.post(stream.url_base, payload=self.request2response(data=self.json_body, headers={"Content-Type": "application/json"})) - assert response["content_type"] == "application/json" - assert json.loads(response["body"]) == self.json_body + response = [] + for r in loop.run_until_complete(read_records(stream)): + response.append(loop.run_until_complete(r)) - def test_text_body(self, mocker, requests_mock): + assert response[0]["content_type"] == "application/json" + assert json.loads(response[0]["body"]) == self.json_body + def test_text_body(self, mocker): stream = PostHttpStream() mocker.patch.object(stream, "request_body_data", return_value=self.data_body) + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) - requests_mock.register_uri("POST", stream.url_base, text=self.request2response) - response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + with aioresponses() as m: + m.post(stream.url_base, payload=self.request2response(data=self.data_body)) - assert response["content_type"] is None - assert response["body"] == self.data_body + response = [] + for r in loop.run_until_complete(read_records(stream)): + response.append(loop.run_until_complete(r)) def test_form_body(self, mocker, requests_mock): From b52c8a190a9832b746530f845bd946f24767923a Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Thu, 28 Dec 2023 20:15:52 -0500 Subject: [PATCH 42/74] Fix caching --- .../sources/streams/http/http_async.py | 6 +- .../sources/streams/http/test_http_async.py | 221 +++++++++++------- 2 files changed, 144 insertions(+), 83 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index c6fe000bc811..f51742189e09 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -79,17 +79,17 @@ def request_session(self) -> aiohttp.ClientSession: sqlite_path = str(Path(cache_dir) / self.cache_filename) else: sqlite_path = "file::memory:?cache=shared" - cache = aiohttp_client_cache.SQLiteBackend(cache_dir=sqlite_path) + cache = aiohttp_client_cache.SQLiteBackend(cache_name=sqlite_path, allowed_methods=("get", "post", "put", "patch", "options", "delete", "list")) return AsyncCachedLimiterSession(cache=cache, connector=connector, api_budget=self._api_budget) else: return AsyncLimiterSession(connector=connector, api_budget=self._api_budget, **kwargs) - def clear_cache(self) -> None: + async def clear_cache(self) -> None: """ Clear cached requests for current session, can be called any time """ if isinstance(self._session, aiohttp_client_cache.CachedSession): - self._session.cache.clear() + await self._session.cache.clear() @abstractmethod async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index 44fc7b80b656..73b9126672cf 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -13,7 +13,7 @@ import aiohttp import pytest import requests -from aioresponses import aioresponses +from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator as HttpTokenAuthenticator @@ -72,9 +72,9 @@ def test_request_kwargs_used(mocker): m.assert_called_once() -async def read_records(stream, sync_mode=SyncMode.full_refresh): +async def read_records(stream, sync_mode=SyncMode.full_refresh, stream_slice=None): records = [] - async for record in stream.read_records(sync_mode=sync_mode): + async for record in stream.read_records(sync_mode=sync_mode, stream_slice=stream_slice): records.append(record) return records @@ -377,29 +377,32 @@ def test_text_body(self, mocker): for r in loop.run_until_complete(read_records(stream)): response.append(loop.run_until_complete(r)) - def test_form_body(self, mocker, requests_mock): + assert response[0]["content_type"] is None + assert response[0]["body"] == self.data_body - stream = PostHttpStream() - mocker.patch.object(stream, "request_body_data", return_value=self.form_body) - - requests_mock.register_uri("POST", stream.url_base, text=self.request2response) - response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] + def test_form_body(self, mocker): + raise NotImplementedError("This is not supported for the async flow yet.") - assert response["content_type"] == "application/x-www-form-urlencoded" - assert response["body"] == self.urlencoded_form_body - - def test_text_json_body(self, mocker, requests_mock): + def test_text_json_body(self, mocker): """checks a exception if both functions were overridden""" stream = PostHttpStream() + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + mocker.patch.object(stream, "request_body_data", return_value=self.data_body) mocker.patch.object(stream, "request_body_json", return_value=self.json_body) - requests_mock.register_uri("POST", stream.url_base, text=self.request2response) - with pytest.raises(RequestBodyException): - list(stream.read_records(sync_mode=SyncMode.full_refresh)) + + with aioresponses() as m: + m.post(stream.url_base, payload=self.request2response(data=self.data_body)) + with pytest.raises(RequestBodyException): + loop.run_until_complete(read_records(stream)) def test_body_for_all_methods(self, mocker, requests_mock): """Stream must send a body for GET/POST/PATCH/PUT methods only""" stream = PostHttpStream() + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + methods = { "POST": True, "PUT": True, @@ -411,12 +414,30 @@ def test_body_for_all_methods(self, mocker, requests_mock): for method, with_body in methods.items(): stream.http_method = method mocker.patch.object(stream, "request_body_data", return_value=self.data_body) - requests_mock.register_uri(method, stream.url_base, text=self.request2response) - response = list(stream.read_records(sync_mode=SyncMode.full_refresh))[0] - if with_body: - assert response["body"] == self.data_body - else: - assert response["body"] is None + + with aioresponses() as m: + if method == "POST": + request = m.post + elif method == "PUT": + request = m.put + elif method == "PATCH": + request = m.patch + elif method == "GET": + request = m.get + elif method == "DELETE": + request = m.delete + elif method == "OPTIONS": + request = m.options + + request(stream.url_base, payload=self.request2response(data=self.data_body)) + + response = [] + for r in loop.run_until_complete(read_records(stream)): + response.append(loop.run_until_complete(r)) + + # The requests library flow strips the body where `with_body` is False, but + # aiohttp does not. + assert response[0]["body"] == self.data_body class CacheHttpStream(StubBasicReadHttpStream): @@ -448,22 +469,14 @@ def test_caching_filename(): def test_caching_sessions_are_different(): stream_1 = CacheHttpStream() stream_2 = CacheHttpStream() + loop = asyncio.get_event_loop() + loop.run_until_complete(stream_1.ensure_session()) + loop.run_until_complete(stream_2.ensure_session()) assert stream_1._session != stream_2._session assert stream_1.cache_filename == stream_2.cache_filename -# def test_cached_streams_wortk_when_request_path_is_not_set(mocker, requests_mock): -# This test verifies that HttpStreams with a cached session work even if the path is not set -# For instance, when running in a unit test -# stream = CacheHttpStream() -# with mocker.patch.object(stream._session, "send", wraps=stream._session.send): -# requests_mock.register_uri("GET", stream.url_base) -# records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) -# assert records == [{"data": 1}] -# "" - - def test_parent_attribute_exist(): parent_stream = CacheHttpStream() child_stream = CacheHttpSubStream(parent=parent_stream) @@ -471,20 +484,25 @@ def test_parent_attribute_exist(): assert child_stream.parent == parent_stream -def test_that_response_was_cached(mocker, requests_mock): - requests_mock.register_uri("GET", "https://google.com/", text="text") +def test_that_response_was_cached(mocker): stream = CacheHttpStream() - stream.clear_cache() + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + loop.run_until_complete(stream.clear_cache()) + mocker.patch.object(stream, "url_base", "https://google.com/") - records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) - assert requests_mock.called + with aioresponses() as m1: + m1.get(stream.url_base) + records = loop.run_until_complete(read_records(stream)) + m1.assert_called_once() - requests_mock.reset_mock() - new_records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) + with aioresponses() as m2: + m2.get(stream.url_base) + new_records = loop.run_until_complete(read_records(stream)) + m2.assert_not_called() assert len(records) == len(new_records) - assert not requests_mock.called class CacheHttpStreamWithSlices(CacheHttpStream): @@ -498,36 +516,51 @@ def stream_slices(self, **kwargs) -> Iterable[Optional[Mapping[str, Any]]]: yield {"path": path} async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: - yield {"value": len(response.text)} + yield {"value": len(await response.text())} @patch("airbyte_cdk.sources.streams.core.logging", MagicMock()) -def test_using_cache(mocker, requests_mock): - requests_mock.register_uri("GET", "https://google.com/", text="text") - requests_mock.register_uri("GET", "https://google.com/search", text="text") - +def test_using_cache(mocker): + raise NotImplementedError("giving up for now") parent_stream = CacheHttpStreamWithSlices() + loop = asyncio.get_event_loop() + loop.run_until_complete(parent_stream.ensure_session()) + loop.run_until_complete(parent_stream.clear_cache()) + mocker.patch.object(parent_stream, "url_base", "https://google.com/") - parent_stream.clear_cache() - assert requests_mock.call_count == 0 - assert len(parent_stream._session.cache.responses) == 0 + call_counter = 0 + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + + with aioresponses() as m: - for _slice in parent_stream.stream_slices(): - list(parent_stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=_slice)) + # assert len(parent_stream._session.cache.responses) == 0 - assert requests_mock.call_count == 2 - assert len(parent_stream._session.cache.responses) == 2 + slices = list(parent_stream.stream_slices()) + m.get(parent_stream.url_base, repeat=True, callback=request_callback, payload=slices[0]) + m.get(parent_stream.url_base, repeat=True, callback=request_callback, payload=slices[1]) - child_stream = CacheHttpSubStream(parent=parent_stream) + assert call_counter == 0 - for _slice in child_stream.stream_slices(sync_mode=SyncMode.full_refresh): - pass - assert requests_mock.call_count == 2 - assert len(parent_stream._session.cache.responses) == 2 - assert parent_stream._session.cache.contains(url="https://google.com/") - assert parent_stream._session.cache.contains(url="https://google.com/search") + r1 = loop.run_until_complete(read_records(parent_stream, stream_slice=slices[0])) + r2 = loop.run_until_complete(read_records(parent_stream, stream_slice=slices[1])) + + assert call_counter == 2 + # assert len(parent_stream._session.cache.responses) == 2 + # + # child_stream = CacheHttpSubStream(parent=parent_stream) + # + # for _slice in child_stream.stream_slices(sync_mode=SyncMode.full_refresh): + # pass + # + # assert call_counter == 2 + # assert len(parent_stream._session.cache.responses) == 2 + # assert parent_stream._session.cache.contains(url="https://google.com/") + # assert parent_stream._session.cache.contains(url="https://google.com/search") + # class AutoFailTrueHttpStream(StubBasicReadHttpStream): @@ -538,15 +571,20 @@ class AutoFailTrueHttpStream(StubBasicReadHttpStream): def test_send_raise_on_http_errors_logs(mocker, status_code): mocker.patch.object(AutoFailTrueHttpStream, "logger") mocker.patch.object(AutoFailTrueHttpStream, "should_retry", mocker.Mock(return_value=False)) + stream = AutoFailTrueHttpStream() - req = requests.PreparedRequest() - res = requests.Response() - res.status_code = status_code - mocker.patch.object(requests.Session, "send", return_value=res) - with pytest.raises(requests.exceptions.HTTPError): - response = stream._send_request(req, {}) - stream.logger.error.assert_called_with(response.text) - assert response.status_code == status_code + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + + req = aiohttp.ClientRequest("GET", URL(stream.url_base)) + + with aioresponses() as m: + m.get(stream.url_base, status=status_code, repeat=True, payload="text") + + with pytest.raises(aiohttp.ClientError): + response = loop.run_until_complete(stream._send_request(req, {})) + stream.logger.error.assert_called_with("text") + assert response.status == status_code @pytest.mark.parametrize( @@ -572,32 +610,53 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): ) def test_default_parse_response_error_message(api_response: dict, expected_message: Optional[str]): stream = StubBasicReadHttpStream() + loop = asyncio.get_event_loop() response = MagicMock() - response.json.return_value = api_response + response.json.return_value = _get_response(api_response) - message = stream.parse_response_error_message(response) + message = loop.run_until_complete(stream.parse_response_error_message(response)) assert message == expected_message -def test_default_parse_response_error_message_not_json(requests_mock): +async def _get_response(response): + return response + + +def test_default_parse_response_error_message_not_json(): stream = StubBasicReadHttpStream() - requests_mock.register_uri("GET", "mock://test.com/not_json", text="this is not json") - response = requests.get("mock://test.com/not_json") + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + + req = aiohttp.ClientRequest("GET", URL("mock://test.com/not_json")) - message = stream.parse_response_error_message(response) + def callback(url, **kwargs): + return CallbackResult(body="this is not json") + + with aioresponses() as m: + m.get("mock://test.com/not_json", callback=callback) + response = loop.run_until_complete(stream._send_request(req, {})) + message = loop.run_until_complete(stream.parse_response_error_message(response)) assert message is None def test_default_get_error_display_message_handles_http_error(mocker): stream = StubBasicReadHttpStream() + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + mocker.patch.object(stream, "parse_response_error_message", return_value="my custom message") - non_http_err_msg = stream.get_error_display_message(RuntimeError("not me")) + non_http_err_msg = loop.run_until_complete(stream.get_error_display_message(RuntimeError("not me"))) assert non_http_err_msg is None - response = requests.Response() - http_exception = requests.HTTPError(response=response) - http_err_msg = stream.get_error_display_message(http_exception) + req = aiohttp.ClientRequest("GET", URL("mock://test.com/not_json")) + + with aioresponses() as m: + m.get("mock://test.com/not_json") + response = loop.run_until_complete(stream._send_request(req, {})) + http_exception = aiohttp.ClientResponseError(request_info=None, history=None, message=response) + + http_err_msg = loop.run_until_complete(stream.get_error_display_message(http_exception)) assert http_err_msg == "my custom message" @@ -692,9 +751,11 @@ def test_duplicate_request_params_are_deduped(deduplicate_query_params, path, pa stream._create_prepared_request(path=path, params=params) else: prepared_request = stream._create_prepared_request(path=path, params=params) - assert prepared_request.url == expected_url + assert str(prepared_request.url) == expected_url def test_connection_pool(): stream = StubBasicReadHttpStream(authenticator=HttpTokenAuthenticator("test-token")) - assert stream._session.adapters["https://"]._pool_connections == 20 + loop = asyncio.get_event_loop() + loop.run_until_complete(stream.ensure_session()) + assert stream._session.connector.limit == 20 From 8f685f01400be144e350c0f01eacb51db70692bc Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 29 Dec 2023 10:54:43 -0500 Subject: [PATCH 43/74] Fix cache test --- .../sources/streams/http/http_async.py | 10 +- .../sources/streams/http/test_http_async.py | 107 ++++++++++++------ 2 files changed, 77 insertions(+), 40 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index f51742189e09..a4e0a3c41c63 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -416,16 +416,14 @@ def __init__(self, parent: AsyncHttpStream, **kwargs: Any): async def stream_slices( self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None ) -> Iterable[Optional[Mapping[str, Any]]]: - parent_stream_slices = self.parent.stream_slices( - sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_state=stream_state - ) - # iterate over all parent stream_slices - async for stream_slice in parent_stream_slices: + async for stream_slice in self.parent.stream_slices( + sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_state=stream_state + ): parent_records = self.parent.read_records( sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_slice=stream_slice, stream_state=stream_state ) # iterate over all parent records with current stream_slice - for record in parent_records: + async for record in parent_records: yield {"parent": record} diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index 73b9126672cf..9601f123f68f 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -7,19 +7,17 @@ import json from http import HTTPStatus from typing import Any, Iterable, Mapping, Optional -from unittest.mock import ANY, MagicMock, patch +from unittest.mock import MagicMock, patch from yarl import URL import aiohttp import pytest -import requests from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator as HttpTokenAuthenticator from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException -from airbyte_cdk.sources.streams.http.requests_native_auth import TokenAuthenticator class StubBasicReadHttpStream(AsyncHttpStream): @@ -31,13 +29,13 @@ def __init__(self, deduplicate_query_params: bool = False, **kwargs): self.resp_counter = 1 self._deduplicate_query_params = deduplicate_query_params - async def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: return None def path(self, **kwargs) -> str: return "" - async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: stubResp = {"data": self.resp_counter} self.resp_counter += 1 yield stubResp @@ -60,7 +58,6 @@ def test_request_kwargs_used(mocker): loop = asyncio.get_event_loop() stream = StubBasicReadHttpStream() loop.run_until_complete(stream.ensure_session()) - request_kwargs = {"chunked": True, "compress": True} mocker.patch.object(stream, "request_kwargs", return_value=request_kwargs) @@ -71,6 +68,8 @@ def test_request_kwargs_used(mocker): m.assert_any_call(stream.url_base, "GET", **request_kwargs) m.assert_called_once() + loop.run_until_complete(stream._session.close()) + async def read_records(stream, sync_mode=SyncMode.full_refresh, stream_slice=None): records = [] @@ -97,7 +96,7 @@ def __init__(self, pages: int = 5): super().__init__() self._pages = pages - async def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: while self.current_page < self._pages: page_token = {"page": self.current_page} self.current_page += 1 @@ -146,7 +145,7 @@ def test_stub_bad_url_http_stream_read_records(): class StubCustomBackoffHttpStream(StubBasicReadHttpStream): - def backoff_time(self, response: requests.Response) -> Optional[float]: + def backoff_time(self, response: aiohttp.ClientResponse) -> Optional[float]: return 0.5 @@ -168,6 +167,7 @@ def request_callback(*args, **kwargs): loop.run_until_complete(read_records(stream)) assert call_counter == stream.max_retries + 1 + loop.run_until_complete(stream._session.close()) @pytest.mark.parametrize("retries", [-20, -1, 0, 1, 2, 10]) @@ -200,6 +200,7 @@ def request_callback(*args, **kwargs): m.assert_called_once() else: assert call_counter == stream.max_retries + 1 + loop.run_until_complete(stream._session.close()) def test_stub_custom_backoff_http_stream_endless_retries(mocker): @@ -232,6 +233,7 @@ def request_callback(*args, **kwargs): loop.run_until_complete(read_records(stream)) assert call_counter == infinite_number + 1 + loop.run_until_complete(stream._session.close()) @pytest.mark.parametrize("http_code", [400, 401, 403]) @@ -246,6 +248,8 @@ def test_4xx_error_codes_http_stream(http_code): with pytest.raises(aiohttp.ClientResponseError): loop.run_until_complete(read_records(stream)) + loop.run_until_complete(stream._session.close()) + class AutoFailFalseHttpStream(StubBasicReadHttpStream): raise_on_http_errors = False @@ -263,6 +267,8 @@ def test_raise_on_http_errors_off_429(): with pytest.raises(DefaultBackoffException): loop.run_until_complete(read_records(stream)) + loop.run_until_complete(stream._session.close()) + @pytest.mark.parametrize("status_code", [500, 501, 503, 504]) def test_raise_on_http_errors_off_5xx(status_code): @@ -281,6 +287,7 @@ def request_callback(*args, **kwargs): loop.run_until_complete(read_records(stream)) assert call_counter == stream.max_retries + 1 + loop.run_until_complete(stream._session.close()) @pytest.mark.parametrize("status_code", [400, 401, 402, 403, 416]) @@ -294,6 +301,7 @@ def test_raise_on_http_errors_off_non_retryable_4xx(status_code): response = loop.run_until_complete(stream._send_request(aiohttp.ClientRequest("GET", URL(stream.url_base)), {})) assert response.status == status_code + loop.run_until_complete(stream._session.close()) @pytest.mark.parametrize( @@ -321,12 +329,13 @@ def request_callback(*args, **kwargs): loop.run_until_complete(read_records(stream)) assert call_counter == stream.max_retries + 1 + loop.run_until_complete(stream._session.close()) class PostHttpStream(StubBasicReadHttpStream): http_method = "POST" - async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: """Returns response data as is""" yield response.json() @@ -363,6 +372,7 @@ def test_json_body(self, mocker): assert response[0]["content_type"] == "application/json" assert json.loads(response[0]["body"]) == self.json_body + loop.run_until_complete(stream._session.close()) def test_text_body(self, mocker): stream = PostHttpStream() @@ -379,6 +389,7 @@ def test_text_body(self, mocker): assert response[0]["content_type"] is None assert response[0]["body"] == self.data_body + loop.run_until_complete(stream._session.close()) def test_form_body(self, mocker): raise NotImplementedError("This is not supported for the async flow yet.") @@ -397,6 +408,8 @@ def test_text_json_body(self, mocker): with pytest.raises(RequestBodyException): loop.run_until_complete(read_records(stream)) + loop.run_until_complete(stream._session.close()) + def test_body_for_all_methods(self, mocker, requests_mock): """Stream must send a body for GET/POST/PATCH/PUT methods only""" stream = PostHttpStream() @@ -439,6 +452,8 @@ def test_body_for_all_methods(self, mocker, requests_mock): # aiohttp does not. assert response[0]["body"] == self.data_body + loop.run_until_complete(stream._session.close()) + class CacheHttpStream(StubBasicReadHttpStream): use_cache = True @@ -451,10 +466,10 @@ class CacheHttpSubStream(AsyncHttpSubStream): def __init__(self, parent): super().__init__(parent=parent) - async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: yield None - def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: return None def path(self, **kwargs) -> str: @@ -475,6 +490,8 @@ def test_caching_sessions_are_different(): assert stream_1._session != stream_2._session assert stream_1.cache_filename == stream_2.cache_filename + loop.run_until_complete(stream_1._session.close()) + loop.run_until_complete(stream_2._session.close()) def test_parent_attribute_exist(): @@ -503,6 +520,7 @@ def test_that_response_was_cached(mocker): m2.assert_not_called() assert len(records) == len(new_records) + loop.run_until_complete(stream._session.close()) class CacheHttpStreamWithSlices(CacheHttpStream): @@ -511,21 +529,26 @@ class CacheHttpStreamWithSlices(CacheHttpStream): def path(self, stream_slice: Mapping[str, Any] = None, **kwargs) -> str: return f'{stream_slice["path"]}' if stream_slice else "" - def stream_slices(self, **kwargs) -> Iterable[Optional[Mapping[str, Any]]]: + async def stream_slices(self, **kwargs) -> Iterable[Optional[Mapping[str, Any]]]: for path in self.paths: yield {"path": path} - async def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: yield {"value": len(await response.text())} +async def stream_slices(stream, sync_mode=SyncMode.full_refresh): + slices = [] + async for s in stream.stream_slices(sync_mode=sync_mode): + slices.append(s) + return slices + + @patch("airbyte_cdk.sources.streams.core.logging", MagicMock()) def test_using_cache(mocker): - raise NotImplementedError("giving up for now") parent_stream = CacheHttpStreamWithSlices() loop = asyncio.get_event_loop() loop.run_until_complete(parent_stream.ensure_session()) - loop.run_until_complete(parent_stream.clear_cache()) mocker.patch.object(parent_stream, "url_base", "https://google.com/") @@ -534,33 +557,44 @@ def request_callback(*args, **kwargs): nonlocal call_counter call_counter += 1 + async def get_urls(stream): + urls = [] + async for u in stream._session.cache.get_urls(): + urls.append(u) + return urls + with aioresponses() as m: + # Set up the mocks + slices = loop.run_until_complete(stream_slices(parent_stream, sync_mode=SyncMode.full_refresh)) + m.get(parent_stream.url_base, callback=request_callback) + m.get(f"{parent_stream.url_base}search", callback=request_callback) - # assert len(parent_stream._session.cache.responses) == 0 + loop.run_until_complete(parent_stream.clear_cache()) + assert call_counter == 0 - slices = list(parent_stream.stream_slices()) - m.get(parent_stream.url_base, repeat=True, callback=request_callback, payload=slices[0]) - m.get(parent_stream.url_base, repeat=True, callback=request_callback, payload=slices[1]) + # Get the parent stream's records; the responses should be cached + loop.run_until_complete(read_records(parent_stream, stream_slice=slices[0])) + loop.run_until_complete(read_records(parent_stream, stream_slice=slices[1])) - assert call_counter == 0 + assert call_counter == 2 + urls = loop.run_until_complete(get_urls(parent_stream)) + assert len(urls) == 2 + child_stream = CacheHttpSubStream(parent=parent_stream) + loop.run_until_complete(child_stream.ensure_session()) - r1 = loop.run_until_complete(read_records(parent_stream, stream_slice=slices[0])) - r2 = loop.run_until_complete(read_records(parent_stream, stream_slice=slices[1])) + # child_stream.stream_slices will call `parent.read_records`, however this shouldn't + # result in a new request to the 3rd party since the response has been cached + loop.run_until_complete(stream_slices(child_stream, sync_mode=SyncMode.full_refresh)) assert call_counter == 2 - # assert len(parent_stream._session.cache.responses) == 2 - # - # child_stream = CacheHttpSubStream(parent=parent_stream) - # - # for _slice in child_stream.stream_slices(sync_mode=SyncMode.full_refresh): - # pass - # - # assert call_counter == 2 - # assert len(parent_stream._session.cache.responses) == 2 - # assert parent_stream._session.cache.contains(url="https://google.com/") - # assert parent_stream._session.cache.contains(url="https://google.com/search") - # + urls = loop.run_until_complete(get_urls(parent_stream)) + assert len(urls) == 2 + assert URL("https://google.com/") in urls + assert URL("https://google.com/search") in urls + + loop.run_until_complete(parent_stream._session.close()) + loop.run_until_complete(child_stream._session.close()) class AutoFailTrueHttpStream(StubBasicReadHttpStream): @@ -586,6 +620,8 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): stream.logger.error.assert_called_with("text") assert response.status == status_code + loop.run_until_complete(stream._session.close()) + @pytest.mark.parametrize( "api_response, expected_message", @@ -637,6 +673,7 @@ def callback(url, **kwargs): response = loop.run_until_complete(stream._send_request(req, {})) message = loop.run_until_complete(stream.parse_response_error_message(response)) assert message is None + loop.run_until_complete(stream._session.close()) def test_default_get_error_display_message_handles_http_error(mocker): @@ -658,6 +695,7 @@ def test_default_get_error_display_message_handles_http_error(mocker): http_err_msg = loop.run_until_complete(stream.get_error_display_message(http_exception)) assert http_err_msg == "my custom message" + loop.run_until_complete(stream._session.close()) @pytest.mark.parametrize( @@ -759,3 +797,4 @@ def test_connection_pool(): loop = asyncio.get_event_loop() loop.run_until_complete(stream.ensure_session()) assert stream._session.connector.limit == 20 + loop.run_until_complete(stream._session.close()) From 55445f2416d8e35add919169b8890e58ddbecfb6 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 29 Dec 2023 15:18:53 -0500 Subject: [PATCH 44/74] All new tests passing except form --- .../sources/abstract_source_async.py | 11 ++++++++ .../sources/test_abstract_source_async.py | 26 ++++++++++++------- 2 files changed, 27 insertions(+), 10 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 507086b2fff4..774eff3e4a92 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -12,11 +12,13 @@ import aiohttp from airbyte_cdk.models import ( + AirbyteConnectionStatus, AirbyteMessage, AirbyteStateMessage, AirbyteStreamStatus, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, + Status, SyncMode, ) from airbyte_cdk.models import Type as MessageType @@ -111,6 +113,15 @@ def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> The error object will be cast to string to display the problem to the user. """ + async def check(self, logger: logging.Logger, config: Mapping[str, Any]) -> AirbyteConnectionStatus: + """Implements the Check Connection operation from the Airbyte Specification. + See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/#check. + """ + check_succeeded, error = await self.check_connection(logger, config) + if not check_succeeded: + return AirbyteConnectionStatus(status=Status.FAILED, message=repr(error)) + return AirbyteConnectionStatus(status=Status.SUCCEEDED) + @abstractmethod def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: """ diff --git a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py b/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py index d9f1e82b3495..5a03f7cd57eb 100644 --- a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py +++ b/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py @@ -2,6 +2,7 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # +import asyncio import copy import datetime import logging @@ -63,7 +64,7 @@ def __init__( self._message_repository = message_repository super().__init__() - def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: if self.check_lambda: return self.check_lambda() return False, "Missing callable." @@ -126,20 +127,23 @@ def message_repository(): def test_successful_check(): """Tests that if a source returns TRUE for the connection check the appropriate connectionStatus success message is returned""" expected = AirbyteConnectionStatus(status=Status.SUCCEEDED) - assert expected == MockSource(check_lambda=lambda: (True, None)).check(logger, {}) + loop = asyncio.get_event_loop() + assert expected == loop.run_until_complete(MockSource(check_lambda=lambda: (True, None)).check(logger, {})) def test_failed_check(): """Tests that if a source returns FALSE for the connection check the appropriate connectionStatus failure message is returned""" expected = AirbyteConnectionStatus(status=Status.FAILED, message="'womp womp'") - assert expected == MockSource(check_lambda=lambda: (False, "womp womp")).check(logger, {}) + loop = asyncio.get_event_loop() + assert expected == loop.run_until_complete(MockSource(check_lambda=lambda: (False, "womp womp")).check(logger, {})) def test_raising_check(mocker): """Tests that if a source raises an unexpected exception the appropriate connectionStatus failure message is returned.""" check_lambda = mocker.Mock(side_effect=BaseException("this should fail")) + loop = asyncio.get_event_loop() with pytest.raises(BaseException): - MockSource(check_lambda=check_lambda).check(logger, {}) + loop.run_until_complete(MockSource(check_lambda=check_lambda).check(logger, {})) class MockStream(AsyncStream): @@ -155,7 +159,7 @@ def __init__( def name(self): return self._name - async def read_records(self, **kwargs) -> Iterable[Mapping[str, Any]]: # type: ignore + async def read_records(self, error: Exception = None, **kwargs) -> Iterable[Mapping[str, Any]]: # type: ignore # Remove None values kwargs = {k: v for k, v in kwargs.items() if v is not None} output_supplied = False @@ -302,14 +306,16 @@ def test_read_stream_emits_repository_message_on_error(mocker, message_repositor assert MESSAGE_FROM_REPOSITORY in messages -async def fake_get_error_display_message(*args, **kwargs): - return "my message" +async def read_records_with_error(*args, **kwargs): + if False: + yield + raise RuntimeError("oh no!") def test_read_stream_with_error_no_display_message(mocker): stream = MockStream(name="my_stream") mocker.patch.object(MockStream, "get_json_schema", return_value={}) - mocker.patch.object(MockStream, "read_records", side_effect=RuntimeError("oh no!")) + stream.read_records = read_records_with_error source = MockSource(streams=[stream]) catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) @@ -321,8 +327,8 @@ def test_read_stream_with_error_no_display_message(mocker): def test_read_stream_with_error_with_display_message(mocker): stream = MockStream(name="my_stream") mocker.patch.object(MockStream, "get_json_schema", return_value={}) - mocker.patch.object(MockStream, "read_records", side_effect=RuntimeError("oh no!")) - mocker.patch.object(stream, "get_error_display_message", fake_get_error_display_message) + stream.read_records = read_records_with_error + stream.get_error_display_message = AsyncMock(return_value="my message") source = MockSource(streams=[stream]) catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) From fcaa5e15f278ef78c980e4fd6f201b334c2bf313 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 29 Dec 2023 15:29:41 -0500 Subject: [PATCH 45/74] Remove rate limiter; it can be added back in later --- .../sources/streams/async_call_rate.py | 455 +----------------- .../sources/streams/http/http_async.py | 6 +- 2 files changed, 5 insertions(+), 456 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py index 40e3a32f43c7..222367ade5dc 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py @@ -5,20 +5,12 @@ import abc import datetime import logging -import time -from datetime import timedelta -from threading import RLock -from typing import TYPE_CHECKING, Any, Mapping, Optional -from urllib import parse +from typing import TYPE_CHECKING, Any import aiohttp import aiohttp_client_cache -from pyrate_limiter import InMemoryBucket, Limiter -from pyrate_limiter import Rate as PyRateRate -from pyrate_limiter import RateItem, TimeClock -from pyrate_limiter.exceptions import BucketFullException -from airbyte_cdk.sources.streams.call_rate import CallRateLimitHit, Rate +from airbyte_cdk.sources.streams.call_rate import AbstractAPIBudget # prevents mypy from complaining about missing session attributes in LimiterMixin if TYPE_CHECKING: @@ -29,449 +21,6 @@ logger = logging.getLogger("airbyte") -class AbstractCallRatePolicy(abc.ABC): - """Call rate policy interface. - Should be configurable with different rules, like N per M for endpoint X. Endpoint X is matched with APIBudget. - """ - - @abc.abstractmethod - def matches(self, request: Any) -> bool: - """Tells if this policy matches specific request and should apply to it - - :param request: - :return: True if policy should apply to this request, False - otherwise - """ - - @abc.abstractmethod - def try_acquire(self, request: Any, weight: int) -> None: - """Try to acquire request - - :param request: a request object representing a single call to API - :param weight: number of requests to deduct from credit - :return: - """ - - @abc.abstractmethod - def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: - """Update call rate counting with current values - - :param available_calls: - :param call_reset_ts: - """ - - -class RequestMatcher(abc.ABC): - """Callable that help to match a request object with call rate policies.""" - - @abc.abstractmethod - def __call__(self, request: Any) -> bool: - """ - - :param request: - :return: True if matches the provided request object, False - otherwise - """ - - -class HttpRequestMatcher(RequestMatcher): - """Simple implementation of RequestMatcher for http requests case""" - - def __init__( - self, - method: Optional[str] = None, - url: Optional[str] = None, - params: Optional[Mapping[str, Any]] = None, - headers: Optional[Mapping[str, Any]] = None, - ): - """Constructor - - :param method: - :param url: - :param params: - :param headers: - """ - self._method = method - self._url = url - self._params = {str(k): str(v) for k, v in (params or {}).items()} - self._headers = {str(k): str(v) for k, v in (headers or {}).items()} - - @staticmethod - def _match_dict(obj: Mapping[str, Any], pattern: Mapping[str, Any]) -> bool: - """Check that all elements from pattern dict present and have the same values in obj dict - - :param obj: - :param pattern: - :return: - """ - return pattern.items() <= obj.items() - - def __call__(self, request: Any) -> bool: - """ - - :param request: - :return: True if matches the provided request object, False - otherwise - """ - # if isinstance(request, requests.Request): # TODO: do I need to handle a non-"prepared" request? - # prepared_request = request.prepare() - if isinstance(request, aiohttp.ClientRequest): - prepared_request = request - else: - return False - - if self._method is not None: - if prepared_request.method != self._method: - return False - if self._url is not None and prepared_request.url is not None: - url_without_params = prepared_request.url.split("?")[0] - if url_without_params != self._url: - return False - if self._params is not None: - parsed_url = parse.urlsplit(prepared_request.url) - params = dict(parse.parse_qsl(str(parsed_url.query))) - if not self._match_dict(params, self._params): - return False - if self._headers is not None: - if not self._match_dict(prepared_request.headers, self._headers): - return False - return True - - -class BaseCallRatePolicy(AbstractCallRatePolicy, abc.ABC): - def __init__(self, matchers: list[RequestMatcher]): - self._matchers = matchers - - def matches(self, request: Any) -> bool: - """Tell if this policy matches specific request and should apply to it - - :param request: - :return: True if policy should apply to this request, False - otherwise - """ - - if not self._matchers: - return True - return any(matcher(request) for matcher in self._matchers) - - -class UnlimitedCallRatePolicy(BaseCallRatePolicy): - """ - This policy is for explicit unlimited call rates. - It can be used when we want to match a specific group of requests and don't apply any limits. - - Example: - - APICallBudget( - [ - UnlimitedCallRatePolicy( - matchers=[HttpRequestMatcher(url="/some/method", headers={"sandbox": true})], - ), - FixedWindowCallRatePolicy( - matchers=[HttpRequestMatcher(url="/some/method")], - next_reset_ts=datetime.now(), - period=timedelta(hours=1) - call_limit=1000, - ), - ] - ) - - The code above will limit all calls to /some/method except calls that have header sandbox=True - """ - - def try_acquire(self, request: Any, weight: int) -> None: - """Do nothing""" - - def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: - """Do nothing""" - - -class FixedWindowCallRatePolicy(BaseCallRatePolicy): - def __init__(self, next_reset_ts: datetime.datetime, period: timedelta, call_limit: int, matchers: list[RequestMatcher]): - """A policy that allows {call_limit} calls within a {period} time interval - - :param next_reset_ts: next call rate reset time point - :param period: call rate reset period - :param call_limit: - :param matchers: - """ - - self._next_reset_ts = next_reset_ts - self._offset = period - self._call_limit = call_limit - self._calls_num = 0 - self._lock = RLock() - super().__init__(matchers=matchers) - - def try_acquire(self, request: Any, weight: int) -> None: - if weight > self._call_limit: - raise ValueError("Weight can not exceed the call limit") - if not self.matches(request): - raise ValueError("Request does not match the policy") - - with self._lock: - self._update_current_window() - - if self._calls_num + weight > self._call_limit: - reset_in = self._next_reset_ts - datetime.datetime.now() - error_message = ( - f"reached maximum number of allowed calls {self._call_limit} " f"per {self._offset} interval, next reset in {reset_in}." - ) - raise CallRateLimitHit( - error=error_message, - item=request, - weight=weight, - rate=f"{self._call_limit} per {self._offset}", - time_to_wait=reset_in, - ) - - self._calls_num += weight - - def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: - """Update call rate counters, by default, only reacts to decreasing updates of available_calls and changes to call_reset_ts. - We ignore updates with available_calls > current_available_calls to support call rate limits that are lower than API limits. - - :param available_calls: - :param call_reset_ts: - """ - with self._lock: - self._update_current_window() - current_available_calls = self._call_limit - self._calls_num - - if available_calls is not None and current_available_calls > available_calls: - logger.debug( - "got rate limit update from api, adjusting available calls from %s to %s", current_available_calls, available_calls - ) - self._calls_num = self._call_limit - available_calls - - if call_reset_ts is not None and call_reset_ts != self._next_reset_ts: - logger.debug("got rate limit update from api, adjusting reset time from %s to %s", self._next_reset_ts, call_reset_ts) - self._next_reset_ts = call_reset_ts - - def _update_current_window(self) -> None: - now = datetime.datetime.now() - if now > self._next_reset_ts: - logger.debug("started new window, %s calls available now", self._call_limit) - self._next_reset_ts = self._next_reset_ts + self._offset - self._calls_num = 0 - - -class MovingWindowCallRatePolicy(BaseCallRatePolicy): - """ - Policy to control requests rate implemented on top of PyRateLimiter lib. - The main difference between this policy and FixedWindowCallRatePolicy is that the rate-limiting window - is moving along requests that we made, and there is no moment when we reset an available number of calls. - This strategy requires saving of timestamps of all requests within a window. - """ - - def __init__(self, rates: list[Rate], matchers: list[RequestMatcher]): - """Constructor - - :param rates: list of rates, the order is important and must be ascending - :param matchers: - """ - if not rates: - raise ValueError("The list of rates can not be empty") - pyrate_rates = [PyRateRate(limit=rate.limit, interval=int(rate.interval.total_seconds() * 1000)) for rate in rates] - self._bucket = InMemoryBucket(pyrate_rates) - # Limiter will create the background task that clears old requests in the bucket - self._limiter = Limiter(self._bucket) - super().__init__(matchers=matchers) - - def try_acquire(self, request: Any, weight: int) -> None: - if not self.matches(request): - raise ValueError("Request does not match the policy") - - try: - self._limiter.try_acquire(request, weight=weight) - except BucketFullException as exc: - item = self._limiter.bucket_factory.wrap_item(request, weight) - assert isinstance(item, RateItem) - - with self._limiter.lock: - time_to_wait = self._bucket.waiting(item) - assert isinstance(time_to_wait, int) - - raise CallRateLimitHit( - error=str(exc.meta_info["error"]), - item=request, - weight=int(exc.meta_info["weight"]), - rate=str(exc.meta_info["rate"]), - time_to_wait=timedelta(milliseconds=time_to_wait), - ) - - def update(self, available_calls: Optional[int], call_reset_ts: Optional[datetime.datetime]) -> None: - """Adjust call bucket to reflect the state of the API server - - :param available_calls: - :param call_reset_ts: - :return: - """ - if available_calls is not None and call_reset_ts is None: # we do our best to sync buckets with API - if available_calls == 0: - with self._limiter.lock: - items_to_add = self._bucket.count() < self._bucket.rates[0].limit - if items_to_add > 0: - now: int = TimeClock().now() # type: ignore[no-untyped-call] - self._bucket.put(RateItem(name="dummy", timestamp=now, weight=items_to_add)) - # TODO: add support if needed, it might be that it is not possible to make a good solution for this case - # if available_calls is not None and call_reset_ts is not None: - # ts = call_reset_ts.timestamp() - - -class AbstractAPIBudget(abc.ABC): - """Interface to some API where a client allowed to have N calls per T interval. - - Important: APIBudget is not doing any API calls, the end user code is responsible to call this interface - to respect call rate limitation of the API. - - It supports multiple policies applied to different group of requests. To distinct these groups we use RequestMatchers. - Individual policy represented by MovingWindowCallRatePolicy and currently supports only moving window strategy. - """ - - @abc.abstractmethod - def acquire_call(self, request: Any, block: bool = True, timeout: Optional[float] = None) -> None: - """Try to get a call from budget, will block by default - - :param request: - :param block: when true (default) will block the current thread until call credit is available - :param timeout: if set will limit maximum time in block, otherwise will wait until credit is available - :raises: CallRateLimitHit - when no credits left and if timeout was set the waiting time exceed the timeout - """ - - @abc.abstractmethod - def get_matching_policy(self, request: Any) -> Optional[AbstractCallRatePolicy]: - """Find matching call rate policy for specific request""" - - @abc.abstractmethod - def update_from_response(self, request: Any, response: Any) -> None: - """Update budget information based on response from API - - :param request: the initial request that triggered this response - :param response: response from the API - """ - - -class APIBudget(AbstractAPIBudget): - """Default APIBudget implementation""" - - def __init__(self, policies: list[AbstractCallRatePolicy], maximum_attempts_to_acquire: int = 100000) -> None: - """Constructor - - :param policies: list of policies in this budget - :param maximum_attempts_to_acquire: number of attempts before throwing hit ratelimit exception, we put some big number here - to avoid situations when many threads compete with each other for a few lots over a significant amount of time - """ - - self._policies = policies - self._maximum_attempts_to_acquire = maximum_attempts_to_acquire - - def get_matching_policy(self, request: Any) -> Optional[AbstractCallRatePolicy]: - for policy in self._policies: - if policy.matches(request): - return policy - return None - - def acquire_call(self, request: Any, block: bool = True, timeout: Optional[float] = None) -> None: - """Try to get a call from budget, will block by default. - Matchers will be called sequentially in the same order they were added. - The first matcher that returns True will - - :param request: - :param block: when true (default) will block the current thread until call credit is available - :param timeout: if provided will limit maximum time in block, otherwise will wait until credit is available - :raises: CallRateLimitHit - when no calls left and if timeout was set the waiting time exceed the timeout - """ - - policy = self.get_matching_policy(request) - if policy: - self._do_acquire(request=request, policy=policy, block=block, timeout=timeout) - elif self._policies: - logger.info("no policies matched with requests, allow call by default") - - def update_from_response(self, request: Any, response: Any) -> None: - """Update budget information based on response from API - - :param request: the initial request that triggered this response - :param response: response from the API - """ - pass - - def _do_acquire(self, request: Any, policy: AbstractCallRatePolicy, block: bool, timeout: Optional[float]) -> None: - """Internal method to try to acquire a call credit - - :param request: - :param policy: - :param block: - :param timeout: - """ - last_exception = None - # sometimes we spend all budget before a second attempt, so we have few more here - for attempt in range(1, self._maximum_attempts_to_acquire): - try: - policy.try_acquire(request, weight=1) - return - except CallRateLimitHit as exc: - last_exception = exc - if block: - if timeout is not None: - time_to_wait = min(timedelta(seconds=timeout), exc.time_to_wait) - else: - time_to_wait = exc.time_to_wait - - time_to_wait = max(timedelta(0), time_to_wait) # sometimes we get negative duration - logger.info("reached call limit %s. going to sleep for %s", exc.rate, time_to_wait) - time.sleep(time_to_wait.total_seconds()) - else: - raise - - if last_exception: - logger.info("we used all %s attempts to acquire and failed", self._maximum_attempts_to_acquire) - raise last_exception - - -class HttpAPIBudget(APIBudget): - """Implementation of AbstractAPIBudget for HTTP""" - - def __init__( - self, - ratelimit_reset_header: str = "ratelimit-reset", - ratelimit_remaining_header: str = "ratelimit-remaining", - status_codes_for_ratelimit_hit: tuple[int] = (429,), - **kwargs: Any, - ): - """Constructor - - :param ratelimit_reset_header: name of the header that has a timestamp of the next reset of call budget - :param ratelimit_remaining_header: name of the header that has the number of calls left - :param status_codes_for_ratelimit_hit: list of HTTP status codes that signal about rate limit being hit - """ - self._ratelimit_reset_header = ratelimit_reset_header - self._ratelimit_remaining_header = ratelimit_remaining_header - self._status_codes_for_ratelimit_hit = status_codes_for_ratelimit_hit - super().__init__(**kwargs) - - def update_from_response(self, request: Any, response: Any) -> None: - policy = self.get_matching_policy(request) - if not policy: - return - - if isinstance(response, aiohttp.ClientResponse): - available_calls = self.get_calls_left_from_response(response) - reset_ts = self.get_reset_ts_from_response(response) - policy.update(available_calls=available_calls, call_reset_ts=reset_ts) - - def get_reset_ts_from_response(self, response: aiohttp.ClientResponse) -> Optional[datetime.datetime]: - if response.headers.get(self._ratelimit_reset_header): - return datetime.datetime.fromtimestamp(int(response.headers[self._ratelimit_reset_header])) - return None - - def get_calls_left_from_response(self, response: aiohttp.ClientResponse) -> Optional[int]: - if response.headers.get(self._ratelimit_remaining_header): - return int(response.headers[self._ratelimit_remaining_header]) - - if response.status in self._status_codes_for_ratelimit_hit: - return 0 - - return None - - class AsyncLimiterMixin(MIXIN_BASE): """Mixin class that adds rate-limiting behavior to requests.""" diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index a4e0a3c41c63..c826ba572a81 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -16,7 +16,6 @@ from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession -from airbyte_cdk.sources.streams.call_rate import APIBudget from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy @@ -40,8 +39,9 @@ class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): Basic building block for users building an Airbyte source for an async HTTP API. """ - def __init__(self, authenticator: Optional[Union[HttpAuthenticator, NoAuth]] = NoAuth(), api_budget: Optional[APIBudget] = None): - self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) + def __init__(self, authenticator: Optional[Union[HttpAuthenticator, NoAuth]] = NoAuth()): + # TODO: wire in rate limiting via ApiBudget + self._api_budget = None self._session: aiohttp.ClientSession = None # TODO: HttpStream handles other authentication codepaths, which may need to be added later self._authenticator = authenticator From fd0dd81990dc834d8ff3c5e134fae1bd81a04b33 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 29 Dec 2023 15:33:05 -0500 Subject: [PATCH 46/74] test_rate_limiting_async.py --- .../utils/test_rate_limiting_async.py | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py diff --git a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py b/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py new file mode 100644 index 000000000000..ad9ea52b4c21 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py @@ -0,0 +1,25 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import aiohttp +import pytest +from airbyte_cdk.sources.streams.http.rate_limiting_async import default_backoff_handler + + +def helper_with_exceptions(exception_type): + raise exception_type + + +@pytest.mark.parametrize( + "max_tries, max_time, factor, exception_to_raise", + [ + (1, None, 1, aiohttp.ServerTimeoutError), + (2, 2, 1, aiohttp.ServerConnectionError), + (2, 2, 1, aiohttp.ServerDisconnectedError), + ], +) +def test_default_backoff_handler(max_tries: int, max_time: int, factor: int, exception_to_raise: Exception): + backoff_handler = default_backoff_handler(max_tries=max_tries, max_time=max_time, factor=factor)(helper_with_exceptions) + with pytest.raises(exception_to_raise): + backoff_handler(exception_to_raise) From ec8f2eda697c4ad4c9f50856987e5d120692d1c3 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 30 Dec 2023 10:41:32 -0500 Subject: [PATCH 47/74] WIP: test_availability_strategy_async.py --- .../http/availability_strategy_async.py | 45 +++-- .../sources/streams/http/http_async.py | 2 +- .../streams/http/rate_limiting_async.py | 1 + .../http/test_availability_strategy_async.py | 170 ++++++++++++++++++ .../sources/streams/http/test_http_async.py | 1 + .../utils/test_rate_limiting_async.py | 1 + 6 files changed, 206 insertions(+), 14 deletions(-) create mode 100644 airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py index 1f724af4db15..93c492820a0f 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py @@ -4,12 +4,13 @@ import logging import typing -from typing import Optional, Tuple +from typing import Dict, Optional, Tuple +import requests +from aiohttp import ClientResponseError from airbyte_cdk.sources.streams import AsyncStream from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy from airbyte_cdk.sources.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice -from requests import HTTPError if typing.TYPE_CHECKING: from airbyte_cdk.sources import Source @@ -39,8 +40,8 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, # without accounting for the case in which the parent stream is empty. reason = f"Cannot attempt to connect to stream {stream.name} - no stream slices were found, likely because the parent stream is empty." return False, reason - except HTTPError as error: - is_available, reason = self.handle_http_error(stream, logger, source, error) + except ClientResponseError as error: + is_available, reason = await self._handle_http_error(stream, logger, source, error) if not is_available: reason = f"Unable to get slices for {stream.name} stream, because of error in parent stream. {reason}" return is_available, reason @@ -51,14 +52,14 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, except StopIteration: logger.info(f"Successfully connected to stream {stream.name}, but got 0 records.") return True, None - except HTTPError as error: - is_available, reason = self.handle_http_error(stream, logger, source, error) + except ClientResponseError as error: + is_available, reason = await self._handle_http_error(stream, logger, source, error) if not is_available: reason = f"Unable to read {stream.name} stream. {reason}" return is_available, reason - async def handle_http_error( - self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: HTTPError + async def _handle_http_error( + self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: ClientResponseError ) -> Tuple[bool, Optional[str]]: """ Override this method to define error handling for various `HTTPError`s @@ -76,7 +77,7 @@ async def handle_http_error( for some reason and the str should describe what went wrong and how to resolve the unavailability, if possible. """ - status_code = error.response.status_code + status_code = error.status known_status_codes = self.reasons_for_unavailable_status_codes(stream, logger, source, error) known_reason = known_status_codes.get(status_code) if not known_reason: @@ -84,8 +85,26 @@ async def handle_http_error( raise error doc_ref = self._visit_docs_message(logger, source) - reason = f"The endpoint {error.response.url} returned {status_code}: {error.response.reason}. {known_reason}. {doc_ref} " - response_error_message = await stream.parse_response_error_message(error.response) - if response_error_message: - reason += response_error_message + reason = f"The endpoint {error.request_info.url} returned {status_code}: {error.message}. {known_reason}. {doc_ref} " return False, reason + + def reasons_for_unavailable_status_codes( + self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: ClientResponseError + ) -> Dict[int, str]: + """ + Returns a dictionary of HTTP status codes that indicate stream + unavailability and reasons explaining why a given status code may + have occurred and how the user can resolve that error, if applicable. + + :param stream: stream + :param logger: source logger + :param source: optional (source) + :return: A dictionary of (status code, reason) where the 'reason' explains + why 'status code' may have occurred and how the user can resolve that + error, if applicable. + """ + reasons_for_codes: Dict[int, str] = { + requests.codes.FORBIDDEN: "This is most likely due to insufficient permissions on the credentials in use. " + "Try to grant required permissions/scopes or re-authenticate" + } + return reasons_for_codes diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index c826ba572a81..3ecadfb73f45 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -54,7 +54,7 @@ def authenticator(self) -> Optional[Union[HttpAuthenticator, NoAuth]]: return self._authenticator @property - async def availability_strategy(self) -> Optional[AsyncHttpAvailabilityStrategy]: + def availability_strategy(self) -> Optional[AsyncHttpAvailabilityStrategy]: return AsyncHttpAvailabilityStrategy() def request_session(self) -> aiohttp.ClientSession: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py index 0567f0115b9a..18721aa0420b 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py @@ -19,6 +19,7 @@ # exceptions.ReadTimeout, # exceptions.ConnectionError, # exceptions.ChunkedEncodingError, + aiohttp.ClientPayloadError, aiohttp.ServerTimeoutError, aiohttp.ServerConnectionError, aiohttp.ServerDisconnectedError, diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py new file mode 100644 index 000000000000..bb703aa67ef1 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py @@ -0,0 +1,170 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# +import asyncio +import logging +from typing import Any, Iterable, List, Mapping, Optional, Tuple + +import aiohttp +import pytest +import requests +from aioresponses import CallbackResult, aioresponses +from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy +from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream + +logger = logging.getLogger("airbyte") + + +class MockHttpStream(AsyncHttpStream): + url_base = "https://test_base_url.com" + primary_key = "" + + def __init__(self, **kwargs): + super().__init__(**kwargs) + self.resp_counter = 1 + + async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + return None + + def path(self, **kwargs) -> str: + return "" + + async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: + stub_resp = {"data": self.resp_counter} + self.resp_counter += 1 + yield stub_resp + + def retry_factor(self) -> float: + return 0.01 + + +@pytest.mark.parametrize( + ("status_code", "expected_is_available", "expected_messages"), + [ + ( + 403, + False, + [ + "This is most likely due to insufficient permissions on the credentials in use.", + ], + ), + (200, True, []), + ], +) +@pytest.mark.parametrize( + ("include_source", "expected_docs_url_messages"), + [ + (True, ["Please visit https://docs.airbyte.com/integrations/sources/MockSource to learn more."]), + (False, ["Please visit the connector's documentation to learn more."]), + ], +) +def test_default_http_availability_strategy( + mocker, + status_code, + expected_is_available, + expected_messages, + include_source, + expected_docs_url_messages, +): + class MockListHttpStream(MockHttpStream): + async def read_records(self, *args, **kvargs): + async for record in super().read_records(*args, **kvargs): + yield record + + http_stream = MockListHttpStream() + assert isinstance(http_stream.availability_strategy, AsyncHttpAvailabilityStrategy) + + class MockSource(AsyncAbstractSource): + def __init__(self, streams: List[Stream] = None): + self._streams = streams + super().__init__() + + async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + return True, "" + + def streams(self, config: Mapping[str, Any]) -> List[Stream]: + if not self._streams: + raise Exception("Stream is not set") + return self._streams + + loop = asyncio.get_event_loop() + loop.run_until_complete(http_stream.ensure_session()) + + with aioresponses() as m: + m.get(http_stream.url_base, status=status_code) + + if include_source: + source = MockSource(streams=[http_stream]) + actual_is_available, reason = loop.run_until_complete(http_stream.check_availability(logger, source)) + else: + actual_is_available, reason = loop.run_until_complete(http_stream.check_availability(logger)) + + assert expected_is_available == actual_is_available + if expected_is_available: + assert reason is None + else: + all_expected_messages = expected_messages + expected_docs_url_messages + for message in all_expected_messages: + assert message in reason + + loop.run_until_complete(http_stream._session.close()) + + +def test_http_availability_raises_unhandled_error(mocker): + http_stream = MockHttpStream() + assert isinstance(http_stream.availability_strategy, AsyncHttpAvailabilityStrategy) + + req = requests.Response() + req.status_code = 404 + mocker.patch.object(requests.Session, "send", return_value=req) + + with pytest.raises(aiohttp): + http_stream.check_availability(logger) + + +def test_send_handles_retries_when_checking_availability(mocker, caplog): + http_stream = MockHttpStream() + assert isinstance(http_stream.availability_strategy, AsyncHttpAvailabilityStrategy) + + req_1 = requests.Response() + req_1.status_code = 429 + req_2 = requests.Response() + req_2.status_code = 503 + req_3 = requests.Response() + req_3.status_code = 200 + mock_send = mocker.patch.object(requests.Session, "send", side_effect=[req_1, req_2, req_3]) + + with caplog.at_level(logging.INFO): + stream_is_available, _ = http_stream.check_availability(logger) + + assert stream_is_available + assert mock_send.call_count == 3 + for message in ["Caught retryable error", "Response Code: 429", "Response Code: 503"]: + assert message in caplog.text + + +@pytest.mark.parametrize("records_as_list", [True, False]) +def test_http_availability_strategy_on_empty_stream(mocker, records_as_list): + class MockEmptyHttpStream(mocker.MagicMock, MockHttpStream): + def __init__(self, *args, **kvargs): + mocker.MagicMock.__init__(self) + self.read_records = mocker.MagicMock() + + empty_stream = MockEmptyHttpStream() + assert isinstance(empty_stream, AsyncHttpStream) + + assert isinstance(empty_stream.availability_strategy, AsyncHttpAvailabilityStrategy) + + # Generator should have no values to generate + if records_as_list: + empty_stream.read_records.return_value = [] + else: + empty_stream.read_records.return_value = iter([]) + + logger = logging.getLogger("airbyte.test-source") + stream_is_available, _ = empty_stream.check_availability(logger) + + assert stream_is_available + assert empty_stream.read_records.called diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index 9601f123f68f..da699c6fdc24 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -307,6 +307,7 @@ def test_raise_on_http_errors_off_non_retryable_4xx(status_code): @pytest.mark.parametrize( "error", ( + aiohttp.ClientPayloadError, aiohttp.ServerDisconnectedError, aiohttp.ServerConnectionError, aiohttp.ServerTimeoutError, diff --git a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py b/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py index ad9ea52b4c21..09f63f56c63a 100644 --- a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py +++ b/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py @@ -14,6 +14,7 @@ def helper_with_exceptions(exception_type): @pytest.mark.parametrize( "max_tries, max_time, factor, exception_to_raise", [ + (1, None, 1, aiohttp.ClientPayloadError), (1, None, 1, aiohttp.ServerTimeoutError), (2, 2, 1, aiohttp.ServerConnectionError), (2, 2, 1, aiohttp.ServerDisconnectedError), From 7f3f48d5fd410cbb4f919ccab2fd777f7c49c0eb Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 30 Dec 2023 13:10:21 -0500 Subject: [PATCH 48/74] Passing: test_availability_strategy_async.py --- .../http/test_availability_strategy_async.py | 67 ++++++++++--------- .../sources/streams/http/test_http_async.py | 2 +- 2 files changed, 37 insertions(+), 32 deletions(-) diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py index bb703aa67ef1..71a8fe4e0a23 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py @@ -7,8 +7,7 @@ import aiohttp import pytest -import requests -from aioresponses import CallbackResult, aioresponses +from aioresponses import aioresponses from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy @@ -61,7 +60,6 @@ def retry_factor(self) -> float: ], ) def test_default_http_availability_strategy( - mocker, status_code, expected_is_available, expected_messages, @@ -116,55 +114,62 @@ def test_http_availability_raises_unhandled_error(mocker): http_stream = MockHttpStream() assert isinstance(http_stream.availability_strategy, AsyncHttpAvailabilityStrategy) - req = requests.Response() - req.status_code = 404 - mocker.patch.object(requests.Session, "send", return_value=req) + loop = asyncio.get_event_loop() + loop.run_until_complete(http_stream.ensure_session()) + + with aioresponses() as m: + m.get(http_stream.url_base, status=404) - with pytest.raises(aiohttp): - http_stream.check_availability(logger) + with pytest.raises(aiohttp.ClientResponseError): + loop.run_until_complete(http_stream.check_availability(logger)) -def test_send_handles_retries_when_checking_availability(mocker, caplog): +def test_send_handles_retries_when_checking_availability(caplog): http_stream = MockHttpStream() assert isinstance(http_stream.availability_strategy, AsyncHttpAvailabilityStrategy) - req_1 = requests.Response() - req_1.status_code = 429 - req_2 = requests.Response() - req_2.status_code = 503 - req_3 = requests.Response() - req_3.status_code = 200 - mock_send = mocker.patch.object(requests.Session, "send", side_effect=[req_1, req_2, req_3]) + loop = asyncio.get_event_loop() + loop.run_until_complete(http_stream.ensure_session()) + + call_counter = 0 - with caplog.at_level(logging.INFO): - stream_is_available, _ = http_stream.check_availability(logger) + def request_callback(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + + with aioresponses() as m: + m.get(http_stream.url_base, status=429, callback=request_callback) + m.get(http_stream.url_base, status=503, callback=request_callback) + m.get(http_stream.url_base, status=200, callback=request_callback) + + with caplog.at_level(logging.INFO): + stream_is_available, _ = loop.run_until_complete(http_stream.check_availability(logger)) assert stream_is_available - assert mock_send.call_count == 3 + assert call_counter == 3 for message in ["Caught retryable error", "Response Code: 429", "Response Code: 503"]: assert message in caplog.text -@pytest.mark.parametrize("records_as_list", [True, False]) -def test_http_availability_strategy_on_empty_stream(mocker, records_as_list): +def test_http_availability_strategy_on_empty_stream(mocker): + empty_stream_called = False + async def empty_aiter(*args, **kwargs): + nonlocal empty_stream_called + empty_stream_called = True + yield + class MockEmptyHttpStream(mocker.MagicMock, MockHttpStream): def __init__(self, *args, **kvargs): mocker.MagicMock.__init__(self) - self.read_records = mocker.MagicMock() + self.read_records = empty_aiter empty_stream = MockEmptyHttpStream() assert isinstance(empty_stream, AsyncHttpStream) - assert isinstance(empty_stream.availability_strategy, AsyncHttpAvailabilityStrategy) - # Generator should have no values to generate - if records_as_list: - empty_stream.read_records.return_value = [] - else: - empty_stream.read_records.return_value = iter([]) - logger = logging.getLogger("airbyte.test-source") - stream_is_available, _ = empty_stream.check_availability(logger) + loop = asyncio.get_event_loop() + stream_is_available, _ = loop.run_until_complete(empty_stream.check_availability(logger)) assert stream_is_available - assert empty_stream.read_records.called + assert empty_stream_called diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index da699c6fdc24..6389ff252e93 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -616,7 +616,7 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): with aioresponses() as m: m.get(stream.url_base, status=status_code, repeat=True, payload="text") - with pytest.raises(aiohttp.ClientError): + with pytest.raises(aiohttp.ClientResponseError): response = loop.run_until_complete(stream._send_request(req, {})) stream.logger.error.assert_called_with("text") assert response.status == status_code From f98f61c312de753fc80468ad6078e28318089412 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 31 Dec 2023 11:02:25 -0500 Subject: [PATCH 49/74] Add availability_strategy_async.py and tests --- .../streams/availability_strategy_async.py | 33 +++++++++ .../test_availability_strategy_async.py | 72 +++++++++++++++++++ 2 files changed, 105 insertions(+) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/availability_strategy_async.py create mode 100644 airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/availability_strategy_async.py new file mode 100644 index 000000000000..3fcc588149ae --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/availability_strategy_async.py @@ -0,0 +1,33 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +import typing +from abc import ABC, abstractmethod +from typing import Optional, Tuple + +from airbyte_cdk.sources.streams import Stream + +if typing.TYPE_CHECKING: + from airbyte_cdk.sources import Source + + +class AsyncAvailabilityStrategy(ABC): + """ + Abstract base class for checking stream availability. + """ + + @abstractmethod + async def check_availability(self, stream: Stream, logger: logging.Logger, source: Optional["Source"]) -> Tuple[bool, Optional[str]]: + """ + Checks stream availability. + + :param stream: stream + :param logger: source logger + :param source: (optional) source + :return: A tuple of (boolean, str). If boolean is true, then the stream + is available, and no str is required. Otherwise, the stream is unavailable + for some reason and the str should describe what went wrong and how to + resolve the unavailability, if possible. + """ diff --git a/airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py new file mode 100644 index 000000000000..8c8d93b09eb2 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py @@ -0,0 +1,72 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import asyncio +import logging +from typing import Any, Iterable, List, Mapping, Optional, Tuple, Union + +from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources import Source +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.availability_strategy_async import AsyncAvailabilityStrategy +from airbyte_cdk.sources.streams.core import StreamData + +logger = logging.getLogger("airbyte") + + +class MockStream(Stream): + def __init__(self, name: str) -> Stream: + self._name = name + + @property + def name(self) -> str: + return self._name + + @property + def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: + pass + + def read_records( + self, + sync_mode: SyncMode, + cursor_field: List[str] = None, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ) -> Iterable[StreamData]: + pass + + +def test_no_availability_strategy(): + stream_1 = MockStream("stream") + assert stream_1.availability_strategy is None + + stream_1_is_available, _ = stream_1.check_availability(logger) + assert stream_1_is_available + + +def test_availability_strategy(): + class MockAvailabilityStrategy(AsyncAvailabilityStrategy): + async def check_availability(self, stream: Stream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, any]: + if stream.name == "available_stream": + return True, None + return False, f"Could not reach stream '{stream.name}'." + + class MockStreamWithAvailabilityStrategy(MockStream): + @property + def availability_strategy(self) -> Optional["AsyncAvailabilityStrategy"]: + return MockAvailabilityStrategy() + + stream_1 = MockStreamWithAvailabilityStrategy("available_stream") + stream_2 = MockStreamWithAvailabilityStrategy("unavailable_stream") + loop = asyncio.get_event_loop() + + for stream in [stream_1, stream_2]: + assert isinstance(stream.availability_strategy, MockAvailabilityStrategy) + + stream_1_is_available, _ = loop.run_until_complete(stream_1.check_availability(logger)) + assert stream_1_is_available + + stream_2_is_available, reason = loop.run_until_complete(stream_2.check_availability(logger)) + assert not stream_2_is_available + assert "Could not reach stream 'unavailable_stream'" in reason From 29af3ed0c10fd5026ef521f3a6fcbc07e87360f1 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Mon, 1 Jan 2024 18:11:28 -0500 Subject: [PATCH 50/74] Reorganize scenario-based test helpers --- .../unit_tests/sources/file_based/test_file_based_scenarios.py | 2 +- .../python/unit_tests/sources/scenario_based/__init__.py | 0 .../python/unit_tests/sources/scenario_based/conftest.py | 0 .../{file_based/test_scenarios.py => scenario_based/helpers.py} | 0 .../streams/concurrent/scenarios/test_concurrent_scenarios.py | 2 +- 5 files changed, 2 insertions(+), 2 deletions(-) create mode 100644 airbyte-cdk/python/unit_tests/sources/scenario_based/__init__.py create mode 100644 airbyte-cdk/python/unit_tests/sources/scenario_based/conftest.py rename airbyte-cdk/python/unit_tests/sources/{file_based/test_scenarios.py => scenario_based/helpers.py} (100%) diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py index 6bc58b8edf96..59d88c8b99d3 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py @@ -126,7 +126,7 @@ wait_for_rediscovery_scenario_multi_stream, wait_for_rediscovery_scenario_single_stream, ) -from unit_tests.sources.file_based.test_scenarios import verify_check, verify_discover, verify_read, verify_spec +from unit_tests.sources.scenario_based.helpers import verify_check, verify_discover, verify_read, verify_spec discover_scenarios = [ csv_multi_stream_scenario, diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/__init__.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/conftest.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/conftest.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/test_scenarios.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py similarity index 100% rename from airbyte-cdk/python/unit_tests/sources/file_based/test_scenarios.py rename to airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py index af2249873035..553b1734e3ff 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py @@ -8,7 +8,7 @@ from _pytest.capture import CaptureFixture from freezegun import freeze_time from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenario -from unit_tests.sources.file_based.test_scenarios import verify_discover, verify_read +from unit_tests.sources.scenario_based.helpers import verify_discover, verify_read from unit_tests.sources.streams.concurrent.scenarios.incremental_scenarios import ( test_incremental_stream_with_slice_boundaries_no_input_state, test_incremental_stream_with_slice_boundaries_with_concurrent_state, From b5122ff8d8a904bbe5bbec9b586128a0f944a67a Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 12:58:11 -0500 Subject: [PATCH 51/74] Revert an unintended change --- .../airbyte_cdk/sources/streams/concurrent/adapters.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py index ec593f1bc0d5..f8a5e3ed65e3 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters.py @@ -24,8 +24,8 @@ from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, NoopCursor from airbyte_cdk.sources.streams.concurrent.default_stream import DefaultStream from airbyte_cdk.sources.streams.concurrent.exceptions import ExceptionWithDisplayMessage -from airbyte_cdk.sources.streams.concurrent.partitions.partition_async import AsyncPartition -from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator_async import AsyncPartitionGenerator +from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition +from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator from airbyte_cdk.sources.streams.concurrent.partitions.record import Record from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.utils.schema_helpers import InternalConfig @@ -255,7 +255,7 @@ def log_stream_sync_configuration(self) -> None: self._abstract_stream.log_stream_sync_configuration() -class StreamPartition(AsyncPartition): +class StreamPartition(Partition): """ This class acts as an adapter between the new Partition interface and the Stream's stream_slice interface @@ -348,7 +348,7 @@ def __repr__(self) -> str: return f"StreamPartition({self._stream.name}, {self._slice})" -class StreamPartitionGenerator(AsyncPartitionGenerator): +class StreamPartitionGenerator(PartitionGenerator): """ This class acts as an adapter between the new PartitionGenerator and Stream.stream_slices @@ -376,7 +376,7 @@ def __init__( self._state = state self._cursor = cursor - def generate(self) -> Iterable[AsyncPartition]: + def generate(self) -> Iterable[Partition]: for s in self._stream.stream_slices(sync_mode=self._sync_mode, cursor_field=self._cursor_field, stream_state=self._state): yield StreamPartition( self._stream, copy.deepcopy(s), self.message_repository, self._sync_mode, self._cursor_field, self._state, self._cursor From abb30f0947981f99269eb1b0ff2c0d17590b214a Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 13:37:11 -0500 Subject: [PATCH 52/74] Reorganize scenario builder --- .../unit_tests/sources/file_based/scenarios/avro_scenarios.py | 2 +- .../unit_tests/sources/file_based/scenarios/check_scenarios.py | 2 +- .../unit_tests/sources/file_based/scenarios/csv_scenarios.py | 2 +- .../sources/file_based/scenarios/file_based_source_builder.py | 2 +- .../sources/file_based/scenarios/incremental_scenarios.py | 2 +- .../unit_tests/sources/file_based/scenarios/jsonl_scenarios.py | 2 +- .../sources/file_based/scenarios/parquet_scenarios.py | 2 +- .../sources/file_based/scenarios/unstructured_scenarios.py | 2 +- .../sources/file_based/scenarios/user_input_schema_scenarios.py | 2 +- .../sources/file_based/scenarios/validation_policy_scenarios.py | 2 +- .../unit_tests/sources/file_based/test_file_based_scenarios.py | 2 +- airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py | 2 +- .../scenarios => scenario_based}/scenario_builder.py | 0 .../streams/concurrent/scenarios/incremental_scenarios.py | 2 +- .../streams/concurrent/scenarios/stream_facade_builder.py | 2 +- .../streams/concurrent/scenarios/stream_facade_scenarios.py | 2 +- .../streams/concurrent/scenarios/test_concurrent_scenarios.py | 2 +- .../scenarios/thread_based_concurrent_stream_scenarios.py | 2 +- .../scenarios/thread_based_concurrent_stream_source_builder.py | 2 +- 19 files changed, 18 insertions(+), 18 deletions(-) rename airbyte-cdk/python/unit_tests/sources/{file_based/scenarios => scenario_based}/scenario_builder.py (100%) diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/avro_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/avro_scenarios.py index f1cdac5838b2..5a2f9da88bf3 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/avro_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/avro_scenarios.py @@ -7,7 +7,7 @@ from unit_tests.sources.file_based.in_memory_files_source import TemporaryAvroFilesStreamReader from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder _single_avro_file = { "a.avro": { diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/check_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/check_scenarios.py index 26136d9cf025..8715dc256d3f 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/check_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/check_scenarios.py @@ -9,7 +9,7 @@ TestErrorOpenFileInMemoryFilesStreamReader, ) from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder _base_success_scenario = ( TestScenarioBuilder() diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py index e6c5824b4e19..186b12c4d4a3 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/csv_scenarios.py @@ -11,7 +11,7 @@ from unit_tests.sources.file_based.helpers import EmptySchemaParser, LowInferenceLimitDiscoveryPolicy from unit_tests.sources.file_based.in_memory_files_source import InMemoryFilesSource from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenario, TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenario, TestScenarioBuilder single_csv_scenario: TestScenario[InMemoryFilesSource] = ( TestScenarioBuilder[InMemoryFilesSource]() diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/file_based_source_builder.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/file_based_source_builder.py index 90deb31fe41b..439fd4f9749a 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/file_based_source_builder.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/file_based_source_builder.py @@ -15,7 +15,7 @@ from airbyte_cdk.sources.file_based.schema_validation_policies import AbstractSchemaValidationPolicy from airbyte_cdk.sources.file_based.stream.cursor import AbstractFileBasedCursor from unit_tests.sources.file_based.in_memory_files_source import InMemoryFilesSource -from unit_tests.sources.file_based.scenarios.scenario_builder import SourceBuilder +from unit_tests.sources.scenario_based.scenario_builder import SourceBuilder class FileBasedSourceBuilder(SourceBuilder[InMemoryFilesSource]): diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/incremental_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/incremental_scenarios.py index 3c3195fbac61..8e96171b8873 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/incremental_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/incremental_scenarios.py @@ -4,7 +4,7 @@ from unit_tests.sources.file_based.helpers import LowHistoryLimitCursor from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import IncrementalScenarioConfig, TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import IncrementalScenarioConfig, TestScenarioBuilder single_csv_input_state_is_earlier_scenario = ( TestScenarioBuilder() diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/jsonl_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/jsonl_scenarios.py index b4a447c4f0c0..55aa4f786a17 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/jsonl_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/jsonl_scenarios.py @@ -7,7 +7,7 @@ from airbyte_cdk.utils.traced_exception import AirbyteTracedException from unit_tests.sources.file_based.helpers import LowInferenceBytesJsonlParser, LowInferenceLimitDiscoveryPolicy from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder single_jsonl_scenario = ( TestScenarioBuilder() diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/parquet_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/parquet_scenarios.py index 0852de4a361a..b9717009da67 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/parquet_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/parquet_scenarios.py @@ -9,7 +9,7 @@ from airbyte_cdk.utils.traced_exception import AirbyteTracedException from unit_tests.sources.file_based.in_memory_files_source import TemporaryParquetFilesStreamReader from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder _single_parquet_file = { "a.parquet": { diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/unstructured_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/unstructured_scenarios.py index f052c4530e4a..ec4a53c88a60 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/unstructured_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/unstructured_scenarios.py @@ -6,7 +6,7 @@ import nltk from airbyte_cdk.utils.traced_exception import AirbyteTracedException from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder # import nltk data for pdf parser nltk.download("punkt") diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/user_input_schema_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/user_input_schema_scenarios.py index 58d528cb7caf..8eaed6a2525d 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/user_input_schema_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/user_input_schema_scenarios.py @@ -7,7 +7,7 @@ from airbyte_cdk.test.catalog_builder import CatalogBuilder from airbyte_protocol.models import SyncMode from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder """ User input schema rules: diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/validation_policy_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/validation_policy_scenarios.py index af1318dba647..05d44ce338a4 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/validation_policy_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/validation_policy_scenarios.py @@ -4,7 +4,7 @@ from airbyte_cdk.sources.file_based.exceptions import FileBasedSourceError from unit_tests.sources.file_based.scenarios.file_based_source_builder import FileBasedSourceBuilder -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder _base_single_stream_scenario = ( TestScenarioBuilder() diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py index 59d88c8b99d3..d8248994e732 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py @@ -97,7 +97,7 @@ single_parquet_scenario, single_partitioned_parquet_scenario, ) -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenario +from unit_tests.sources.scenario_based.scenario_builder import TestScenario from unit_tests.sources.file_based.scenarios.unstructured_scenarios import ( corrupted_file_scenario, no_file_extension_unstructured_scenario, diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py index 747d22a31a1f..a0e9d4f75d60 100644 --- a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py +++ b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py @@ -17,7 +17,7 @@ from airbyte_cdk.test.entrypoint_wrapper import read as entrypoint_read from airbyte_cdk.utils.traced_exception import AirbyteTracedException from airbyte_protocol.models import AirbyteLogMessage, AirbyteMessage, ConfiguredAirbyteCatalog -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenario +from unit_tests.sources.scenario_based.scenario_builder import TestScenario def verify_discover(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/scenarios/scenario_builder.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/scenario_builder.py similarity index 100% rename from airbyte-cdk/python/unit_tests/sources/file_based/scenarios/scenario_builder.py rename to airbyte-cdk/python/unit_tests/sources/scenario_based/scenario_builder.py diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/incremental_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/incremental_scenarios.py index 72a0425bc098..6fd76b710387 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/incremental_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/incremental_scenarios.py @@ -3,7 +3,7 @@ # from airbyte_cdk.sources.streams.concurrent.cursor import CursorField from airbyte_cdk.sources.streams.concurrent.state_converters.abstract_stream_state_converter import ConcurrencyCompatibleStateType -from unit_tests.sources.file_based.scenarios.scenario_builder import IncrementalScenarioConfig, TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import IncrementalScenarioConfig, TestScenarioBuilder from unit_tests.sources.streams.concurrent.scenarios.stream_facade_builder import StreamFacadeSourceBuilder from unit_tests.sources.streams.concurrent.scenarios.utils import MockStream diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py index 716eb5508eaf..dc7724b6ba50 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_builder.py @@ -16,7 +16,7 @@ from airbyte_cdk.sources.streams.concurrent.cursor import ConcurrentCursor, CursorField, NoopCursor from airbyte_cdk.sources.streams.concurrent.state_converters.datetime_stream_state_converter import EpochValueConcurrentStreamStateConverter from airbyte_protocol.models import ConfiguredAirbyteStream -from unit_tests.sources.file_based.scenarios.scenario_builder import SourceBuilder +from unit_tests.sources.scenario_based.scenario_builder import SourceBuilder from unit_tests.sources.streams.concurrent.scenarios.thread_based_concurrent_stream_source_builder import NeverLogSliceLogger _CURSOR_FIELD = "cursor_field" diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_scenarios.py index ae66d3a44374..d780036b2ccf 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/stream_facade_scenarios.py @@ -2,7 +2,7 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # from airbyte_cdk.sources.streams.concurrent.cursor import CursorField -from unit_tests.sources.file_based.scenarios.scenario_builder import IncrementalScenarioConfig, TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import IncrementalScenarioConfig, TestScenarioBuilder from unit_tests.sources.streams.concurrent.scenarios.stream_facade_builder import StreamFacadeSourceBuilder from unit_tests.sources.streams.concurrent.scenarios.utils import MockStream diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py index 553b1734e3ff..d52c913778dc 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py @@ -7,7 +7,7 @@ import pytest from _pytest.capture import CaptureFixture from freezegun import freeze_time -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenario +from unit_tests.sources.scenario_based.scenario_builder import TestScenario from unit_tests.sources.scenario_based.helpers import verify_discover, verify_read from unit_tests.sources.streams.concurrent.scenarios.incremental_scenarios import ( test_incremental_stream_with_slice_boundaries_no_input_state, diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_scenarios.py index 2f4ab9b9fccb..8828b599acbb 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_scenarios.py @@ -6,7 +6,7 @@ from airbyte_cdk.sources.message import InMemoryMessageRepository from airbyte_cdk.sources.streams.concurrent.default_stream import DefaultStream from airbyte_cdk.sources.streams.concurrent.partitions.record import Record -from unit_tests.sources.file_based.scenarios.scenario_builder import TestScenarioBuilder +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder from unit_tests.sources.streams.concurrent.scenarios.thread_based_concurrent_stream_source_builder import ( AlwaysAvailableAvailabilityStrategy, ConcurrentSourceBuilder, diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py index 943aea30dbba..45290ff73ba2 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py @@ -20,7 +20,7 @@ from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.utils.slice_logger import SliceLogger from airbyte_protocol.models import ConfiguredAirbyteStream -from unit_tests.sources.file_based.scenarios.scenario_builder import SourceBuilder +from unit_tests.sources.scenario_based.scenario_builder import SourceBuilder class LegacyStream(Stream): From b47821b0c87aaaad8f35029409e28ff97bcaed14 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 15:34:04 -0500 Subject: [PATCH 53/74] Remove capsys from scenario-based tests as it's flaky; instead, iterate over a new `get_source_iter` in entrypoint (now called by `launch`) --- airbyte-cdk/python/airbyte_cdk/entrypoint.py | 10 +++-- .../file_based/test_file_based_scenarios.py | 12 +++--- .../sources/scenario_based/conftest.py | 0 .../sources/scenario_based/helpers.py | 41 +++++++++---------- .../scenarios/test_concurrent_scenarios.py | 4 +- 5 files changed, 34 insertions(+), 33 deletions(-) delete mode 100644 airbyte-cdk/python/unit_tests/sources/scenario_based/conftest.py diff --git a/airbyte-cdk/python/airbyte_cdk/entrypoint.py b/airbyte-cdk/python/airbyte_cdk/entrypoint.py index 6ee464eef57d..e2fb56a82a59 100644 --- a/airbyte-cdk/python/airbyte_cdk/entrypoint.py +++ b/airbyte-cdk/python/airbyte_cdk/entrypoint.py @@ -207,11 +207,15 @@ def _emit_queued_messages(self, source: Source) -> Iterable[AirbyteMessage]: return -def launch(source: Source, args: List[str]) -> None: +def get_source_iter(source: Source, args: List[str]) -> Iterable[str]: source_entrypoint = AirbyteEntrypoint(source) parsed_args = source_entrypoint.parse_args(args) - for message in source_entrypoint.run(parsed_args): - print(message[:50]) + return source_entrypoint.run(parsed_args) + + +def launch(source: Source, args: List[str]) -> None: + for message in get_source_iter(source, args): + print(message) def _init_internal_request_filter() -> None: diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py index d8248994e732..2281c1c8c05b 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py @@ -247,8 +247,8 @@ @pytest.mark.parametrize("scenario", discover_scenarios, ids=[s.name for s in discover_scenarios]) -def test_file_based_discover(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: - verify_discover(capsys, tmp_path, scenario) +def test_file_based_discover(tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: + verify_discover(tmp_path, scenario) @pytest.mark.parametrize("scenario", read_scenarios, ids=[s.name for s in read_scenarios]) @@ -258,10 +258,10 @@ def test_file_based_read(scenario: TestScenario[AbstractSource]) -> None: @pytest.mark.parametrize("scenario", spec_scenarios, ids=[c.name for c in spec_scenarios]) -def test_file_based_spec(capsys: CaptureFixture[str], scenario: TestScenario[AbstractSource]) -> None: - verify_spec(capsys, scenario) +def test_file_based_spec(scenario: TestScenario[AbstractSource]) -> None: + verify_spec(scenario) @pytest.mark.parametrize("scenario", check_scenarios, ids=[c.name for c in check_scenarios]) -def test_file_based_check(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: - verify_check(capsys, tmp_path, scenario) +def test_file_based_check(tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: + verify_check(tmp_path, scenario) diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/conftest.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/conftest.py deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py index a0e9d4f75d60..7d38ef1d5105 100644 --- a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py +++ b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py @@ -8,9 +8,9 @@ from typing import Any, Dict, List, Mapping, Optional, Union import pytest -from _pytest.capture import CaptureFixture from _pytest.reports import ExceptionInfo -from airbyte_cdk.entrypoint import launch +from airbyte_cdk.entrypoint import get_source_iter +from airbyte_cdk.logger import AirbyteLogFormatter from airbyte_cdk.models import AirbyteAnalyticsTraceMessage, SyncMode from airbyte_cdk.sources import AbstractSource from airbyte_cdk.test.entrypoint_wrapper import EntrypointOutput @@ -20,16 +20,16 @@ from unit_tests.sources.scenario_based.scenario_builder import TestScenario -def verify_discover(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: +def verify_discover(tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: expected_exc, expected_msg = scenario.expected_discover_error expected_logs = scenario.expected_logs if expected_exc: with pytest.raises(expected_exc) as exc: - discover(capsys, tmp_path, scenario) + discover(tmp_path, scenario) if expected_msg: assert expected_msg in get_error_message_from_exc(exc) elif scenario.expected_catalog: - output = discover(capsys, tmp_path, scenario) + output = discover(tmp_path, scenario) catalog, logs = output["catalog"], output["logs"] assert catalog == scenario.expected_catalog if expected_logs: @@ -117,50 +117,47 @@ def _verify_expected_logs(logs: List[AirbyteLogMessage], expected_logs: Optional assert expected_message in actual_message -def verify_spec(capsys: CaptureFixture[str], scenario: TestScenario[AbstractSource]) -> None: - assert spec(capsys, scenario) == scenario.expected_spec +def verify_spec(scenario: TestScenario[AbstractSource]) -> None: + assert spec(scenario) == scenario.expected_spec -def verify_check(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: +def verify_check(tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> None: expected_exc, expected_msg = scenario.expected_check_error if expected_exc: with pytest.raises(expected_exc): - output = check(capsys, tmp_path, scenario) + output = check(tmp_path, scenario) if expected_msg: # expected_msg is a string. what's the expected value field? assert expected_msg.value in output["message"] # type: ignore assert output["status"] == scenario.expected_check_status else: - output = check(capsys, tmp_path, scenario) + output = check(tmp_path, scenario) assert output["status"] == scenario.expected_check_status -def spec(capsys: CaptureFixture[str], scenario: TestScenario[AbstractSource]) -> Mapping[str, Any]: - launch( +def spec(scenario: TestScenario[AbstractSource]) -> Mapping[str, Any]: + output = get_source_iter( scenario.source, ["spec"], ) - captured = capsys.readouterr() - return json.loads(captured.out.splitlines()[0])["spec"] # type: ignore + return json.loads(next(output))["spec"] # type: ignore -def check(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> Dict[str, Any]: - launch( +def check(tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> Dict[str, Any]: + output = get_source_iter( scenario.source, ["check", "--config", make_file(tmp_path / "config.json", scenario.config)], ) - captured = capsys.readouterr() - return json.loads(captured.out.splitlines()[0])["connectionStatus"] # type: ignore + return json.loads(next(output))["connectionStatus"] # type: ignore -def discover(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> Dict[str, Any]: - launch( +def discover(tmp_path: PosixPath, scenario: TestScenario[AbstractSource]) -> Dict[str, Any]: + output = [json.loads(o) for o in get_source_iter( scenario.source, ["discover", "--config", make_file(tmp_path / "config.json", scenario.config)], - ) - output = [json.loads(line) for line in capsys.readouterr().out.splitlines()] + )] [catalog] = [o["catalog"] for o in output if o.get("catalog")] # type: ignore return { "catalog": catalog, diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py index d52c913778dc..9d24ce9e5288 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py @@ -72,5 +72,5 @@ def test_concurrent_read(scenario: TestScenario) -> None: @pytest.mark.parametrize("scenario", scenarios, ids=[s.name for s in scenarios]) -def test_concurrent_discover(capsys: CaptureFixture[str], tmp_path: PosixPath, scenario: TestScenario) -> None: - verify_discover(capsys, tmp_path, scenario) +def test_concurrent_discover(tmp_path: PosixPath, scenario: TestScenario) -> None: + verify_discover(tmp_path, scenario) From 5722338156db209fd97e5ab685d14af5ed91f35c Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 15:35:17 -0500 Subject: [PATCH 54/74] Add scenario-based tests for asyncio-based concurrency --- .../async_concurrent_stream_scenarios.py | 342 ++++++++++++++++++ .../test_async_concurrent_scenarios.py | 57 +++ .../async_concurrent_stream_source_builder.py | 116 ++++++ 3 files changed, 515 insertions(+) create mode 100644 airbyte-cdk/python/unit_tests/sources/concurrent_source/async_concurrent_stream_scenarios.py create mode 100644 airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py create mode 100644 airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/async_concurrent_stream_source_builder.py diff --git a/airbyte-cdk/python/unit_tests/sources/concurrent_source/async_concurrent_stream_scenarios.py b/airbyte-cdk/python/unit_tests/sources/concurrent_source/async_concurrent_stream_scenarios.py new file mode 100644 index 000000000000..09bb29629e3c --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/concurrent_source/async_concurrent_stream_scenarios.py @@ -0,0 +1,342 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# +import logging + +from airbyte_cdk.sources.message import InMemoryMessageRepository +from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder +from unit_tests.sources.streams.concurrent.scenarios.async_concurrent_stream_source_builder import ( + AlwaysAvailableAvailabilityStrategy, + ConcurrentSourceBuilder, + LocalAsyncStream, +) + +_id_only_stream = LocalAsyncStream( + name="stream1", + json_schema={ + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=[], + cursor_field=None, + slices=[[{"id": "1"}, {"id": "2"}]], +) + +_id_only_stream_with_primary_key = LocalAsyncStream( + name="stream1", + json_schema={ + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=["id"], + cursor_field=None, + slices=[[{"id": "1"}, {"id": "2"}]], +) + +_id_only_stream_multiple_partitions = LocalAsyncStream( + name="stream1", + json_schema={ + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=[], + cursor_field=None, + slices=[[{"id": "1"}, {"id": "2"}], [{"id": "3"}, {"id": "4"}]], +) + +_id_only_stream_multiple_partitions_concurrency_level_two = LocalAsyncStream( # TODO: allow concurrency level to be set + name="stream1", + json_schema={ + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=[], + cursor_field=None, + slices=[[{"id": "1"}, {"id": "2"}], [{"id": "3"}, {"id": "4"}]], +) + +_stream_raising_exception = LocalAsyncStream( + name="stream1", + json_schema={ + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=[], + cursor_field=None, + slices=[[{"id": "1"}, ValueError("test exception")]], +) + +test_concurrent_cdk_single_stream = ( + TestScenarioBuilder() + .set_name("test_concurrent_cdk_single_stream") + .set_config({}) + .set_source_builder( + ConcurrentSourceBuilder() + .set_streams( + [ + _id_only_stream, + ] + ) + .set_message_repository(InMemoryMessageRepository()) + ) + .set_expected_records( + [ + {"data": {"id": "1"}, "stream": "stream1"}, + {"data": {"id": "2"}, "stream": "stream1"}, + ] + ) + .set_expected_catalog( + { + "streams": [ + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + "name": "stream1", + "supported_sync_modes": ["full_refresh"], + } + ] + } + ) + .build() +) + +test_concurrent_cdk_single_stream_with_primary_key = ( + TestScenarioBuilder() + .set_name("test_concurrent_cdk_single_stream_with_primary_key") + .set_config({}) + .set_source_builder( + ConcurrentSourceBuilder() + .set_streams( + [ + _id_only_stream_with_primary_key, + ] + ) + .set_message_repository(InMemoryMessageRepository()) + ) + .set_expected_records( + [ + {"data": {"id": "1"}, "stream": "stream1"}, + {"data": {"id": "2"}, "stream": "stream1"}, + ] + ) + .set_expected_catalog( + { + "streams": [ + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + "name": "stream1", + "supported_sync_modes": ["full_refresh"], + "source_defined_primary_key": [["id"]], + } + ] + } + ) + .build() +) + +test_concurrent_cdk_multiple_streams = ( + TestScenarioBuilder() + .set_name("test_concurrent_cdk_multiple_streams") + .set_config({}) + .set_source_builder( + ConcurrentSourceBuilder() + .set_streams( + [ + _id_only_stream, + LocalAsyncStream( + name="stream2", + json_schema={ + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + "key": {"type": ["null", "string"]}, + }, + }, + availability_strategy=AlwaysAvailableAvailabilityStrategy(), + primary_key=[], + cursor_field=None, + slices=[[{"id": "10", "key": "v1"}, {"id": "20", "key": "v2"}]], + ), + ] + ) + .set_message_repository(InMemoryMessageRepository()) + ) + .set_expected_records( + [ + {"data": {"id": "1"}, "stream": "stream1"}, + {"data": {"id": "2"}, "stream": "stream1"}, + {"data": {"id": "10", "key": "v1"}, "stream": "stream2"}, + {"data": {"id": "20", "key": "v2"}, "stream": "stream2"}, + ] + ) + .set_expected_catalog( + { + "streams": [ + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + "name": "stream1", + "supported_sync_modes": ["full_refresh"], + }, + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + "key": {"type": ["null", "string"]}, + }, + }, + "name": "stream2", + "supported_sync_modes": ["full_refresh"], + }, + ] + } + ) + .build() +) + +test_concurrent_cdk_partition_raises_exception = ( + TestScenarioBuilder() + .set_name("test_concurrent_partition_raises_exception") + .set_config({}) + .set_source_builder( + ConcurrentSourceBuilder() + .set_streams( + [ + _stream_raising_exception, + ] + ) + .set_message_repository(InMemoryMessageRepository()) + ) + .set_expected_records( + [ + {"data": {"id": "1"}, "stream": "stream1"}, + ] + ) + .set_expected_read_error(ValueError, "test exception") + .set_expected_catalog( + { + "streams": [ + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + "name": "stream1", + "supported_sync_modes": ["full_refresh"], + } + ] + } + ) + .build() +) + +test_concurrent_cdk_single_stream_multiple_partitions = ( + TestScenarioBuilder() + .set_name("test_concurrent_cdk_single_stream_multiple_partitions") + .set_config({}) + .set_source_builder( + ConcurrentSourceBuilder() + .set_streams( + [ + _id_only_stream_multiple_partitions, + ] + ) + .set_message_repository(InMemoryMessageRepository()) + ) + .set_expected_records( + [ + {"data": {"id": "1"}, "stream": "stream1"}, + {"data": {"id": "2"}, "stream": "stream1"}, + {"data": {"id": "3"}, "stream": "stream1"}, + {"data": {"id": "4"}, "stream": "stream1"}, + ] + ) + .set_expected_catalog( + { + "streams": [ + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + "name": "stream1", + "supported_sync_modes": ["full_refresh"], + } + ] + } + ) + .build() +) + +test_concurrent_cdk_single_stream_multiple_partitions_concurrency_level_two = ( + TestScenarioBuilder() + .set_name("test_concurrent_cdk_single_stream_multiple_partitions_concurrency_level_2") + .set_config({}) + .set_source_builder( + ConcurrentSourceBuilder() + .set_streams( + [ + _id_only_stream_multiple_partitions_concurrency_level_two, + ] + ) + .set_message_repository(InMemoryMessageRepository()) + ) + .set_expected_records( + [ + {"data": {"id": "1"}, "stream": "stream1"}, + {"data": {"id": "2"}, "stream": "stream1"}, + {"data": {"id": "3"}, "stream": "stream1"}, + {"data": {"id": "4"}, "stream": "stream1"}, + ] + ) + .set_expected_catalog( + { + "streams": [ + { + "json_schema": { + "type": "object", + "properties": { + "id": {"type": ["null", "string"]}, + }, + }, + "name": "stream1", + "supported_sync_modes": ["full_refresh"], + } + ] + } + ) + .build() +) diff --git a/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py new file mode 100644 index 000000000000..a98d088a5b61 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py @@ -0,0 +1,57 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +from pathlib import PosixPath + +import pytest +from _pytest.capture import CaptureFixture +from freezegun import freeze_time +from pytest import LogCaptureFixture +from unit_tests.sources.scenario_based.helpers import verify_discover, verify_read +from unit_tests.sources.scenario_based.scenario_builder import TestScenario +# from unit_tests.sources.streams.concurrent.scenarios.async_incremental_scenarios import ( +# test_incremental_stream_with_slice_boundaries_no_input_state, +# test_incremental_stream_with_slice_boundaries_with_concurrent_state, +# test_incremental_stream_with_slice_boundaries_with_legacy_state, +# test_incremental_stream_without_slice_boundaries_no_input_state, +# test_incremental_stream_without_slice_boundaries_with_concurrent_state, +# test_incremental_stream_without_slice_boundaries_with_legacy_state, +# ) +from unit_tests.sources.concurrent_source.async_concurrent_stream_scenarios import ( + test_concurrent_cdk_multiple_streams, + test_concurrent_cdk_partition_raises_exception, + test_concurrent_cdk_single_stream, + test_concurrent_cdk_single_stream_multiple_partitions, + test_concurrent_cdk_single_stream_multiple_partitions_concurrency_level_two, + test_concurrent_cdk_single_stream_with_primary_key, +) + +scenarios = [ + test_concurrent_cdk_single_stream, + test_concurrent_cdk_multiple_streams, + test_concurrent_cdk_single_stream_multiple_partitions, + test_concurrent_cdk_single_stream_multiple_partitions_concurrency_level_two, + test_concurrent_cdk_single_stream_with_primary_key, + test_concurrent_cdk_partition_raises_exception, + # test_incremental_stream_with_slice_boundaries, + # test_incremental_stream_without_slice_boundaries, + # test_incremental_stream_with_many_slices_but_without_slice_boundaries, + # test_incremental_stream_with_slice_boundaries_no_input_state, + # test_incremental_stream_with_slice_boundaries_with_concurrent_state, + # test_incremental_stream_with_slice_boundaries_with_legacy_state, + # test_incremental_stream_without_slice_boundaries_no_input_state, + # test_incremental_stream_without_slice_boundaries_with_concurrent_state, + # test_incremental_stream_without_slice_boundaries_with_legacy_state, +] + + +@pytest.mark.parametrize("scenario", scenarios, ids=[s.name for s in scenarios]) +@freeze_time("2023-06-09T00:00:00Z") +def test_concurrent_read(caplog: LogCaptureFixture, tmp_path: PosixPath, scenario: TestScenario) -> None: + verify_read(caplog, tmp_path, scenario) + + +@pytest.mark.parametrize("scenario", scenarios, ids=[s.name for s in scenarios]) +def test_concurrent_discover(tmp_path: PosixPath, scenario: TestScenario) -> None: + verify_discover(tmp_path, scenario) diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/async_concurrent_stream_source_builder.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/async_concurrent_stream_source_builder.py new file mode 100644 index 000000000000..24604b2c2204 --- /dev/null +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/async_concurrent_stream_source_builder.py @@ -0,0 +1,116 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# +import logging +from typing import Any, Dict, Iterable, List, Mapping, Optional, Tuple, Union + +from airbyte_cdk.models import ConfiguredAirbyteCatalog, ConnectorSpecification, DestinationSyncMode, SyncMode +from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.message import MessageRepository +from airbyte_cdk.sources.streams.core_async import AsyncStream +from airbyte_cdk.sources.streams.availability_strategy_async import AsyncAvailabilityStrategy +from airbyte_cdk.sources.streams.core import StreamData +from airbyte_protocol.models import ConfiguredAirbyteStream +from unit_tests.sources.scenario_based.scenario_builder import SourceBuilder + + +class AsyncConcurrentCdkSource(AsyncAbstractSource): + def __init__(self, streams: List[AsyncStream]): + self._streams = streams + super().__init__() + + async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + # Check is not verified because it is up to the source to implement this method + return True, None + + def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: + return self._streams + + def spec(self, *args: Any, **kwargs: Any) -> ConnectorSpecification: + return ConnectorSpecification(connectionSpecification={}) + + def read_catalog(self, catalog_path: str) -> ConfiguredAirbyteCatalog: + return ConfiguredAirbyteCatalog( + streams=[ + ConfiguredAirbyteStream( + stream=s.as_airbyte_stream(), + sync_mode=SyncMode.full_refresh, + destination_sync_mode=DestinationSyncMode.overwrite, + ) + for s in self._streams + ] + ) + + +class ConcurrentSourceBuilder(SourceBuilder[AsyncConcurrentCdkSource]): + def __init__(self): + self._streams: List[AsyncStream] = [] + self._message_repository = None + + def build(self, configured_catalog: Optional[Mapping[str, Any]]) -> AsyncConcurrentCdkSource: + return AsyncConcurrentCdkSource(self._streams) + + def set_streams(self, streams: List[AsyncStream]) -> "ConcurrentSourceBuilder": + self._streams = streams + return self + + def set_message_repository(self, message_repository: MessageRepository) -> "ConcurrentSourceBuilder": + self._message_repository = message_repository + return self + + +class AlwaysAvailableAvailabilityStrategy(AsyncAvailabilityStrategy): + async def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"]) -> Tuple[bool, Optional[str]]: + return True, None + + +class LocalAsyncStream(AsyncStream): + def __init__( + self, + name: str, + json_schema: Mapping[str, Any], + availability_strategy: Optional[AsyncAvailabilityStrategy], + primary_key: Any, # TODO + cursor_field: Any, # TODO + slices: List[List[Mapping[str, Any]]] + ): + self._name = name + self._json_schema = json_schema + self._availability_strategy = availability_strategy + self._primary_key = primary_key + self._cursor_field = cursor_field + self._slices = slices + + @property + def name(self): + return self._name + + async def read_records( + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_slice: Optional[Mapping[str, Any]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> Iterable[StreamData]: + if stream_slice: + for record in stream_slice: + yield record + else: + raise NotImplementedError + + async def stream_slices( + self, *, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None + ) -> Iterable[Optional[Mapping[str, Any]]]: + for stream_slice in self._slices: + yield stream_slice + + @property + def availability_strategy(self) -> Optional[AsyncAvailabilityStrategy]: + return self._availability_strategy + + @property + def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: + return self._primary_key + + def get_json_schema(self) -> Mapping[str, Any]: + return self._json_schema From 1dda38861ab8f1c45a01598542b9a19cc482e23b Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 15:57:55 -0500 Subject: [PATCH 55/74] wip --- .../python/airbyte_cdk/sources/abstract_source_async.py | 4 ++-- .../sources/streams/concurrent/default_stream_async.py | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 774eff3e4a92..88dce2a344f7 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -151,7 +151,7 @@ def read( n_records += 1 yield record - print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") + # print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") logger.info(f"Finished syncing {self.name}") def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): @@ -245,7 +245,7 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s finally: timer.finish_event() logger.info(f"Finished syncing {configured_stream.stream.name}") - logger.info(timer.report()) + # logger.info(timer.report()) # TODO - this is causing scenario-based test failures async def _read_stream( self, diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py index 86b86f033a1e..9b48dae873f4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py @@ -7,13 +7,13 @@ from typing import Any, Iterable, List, Mapping, Optional from airbyte_cdk.models import AirbyteStream, SyncMode -from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream +from airbyte_cdk.sources.streams.concurrent.abstract_stream_async import AsyncAbstractStream from airbyte_cdk.sources.streams.concurrent.availability_strategy import AbstractAvailabilityStrategy, StreamAvailability from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator -class AsyncDefaultStream(AbstractStream): +class AsyncDefaultStream(AsyncAbstractStream): def __init__( self, partition_generator: PartitionGenerator, @@ -42,7 +42,7 @@ async def generate_partitions(self) -> Iterable[Partition]: def name(self) -> str: return self._name - def check_availability(self) -> StreamAvailability: + async def check_availability(self) -> StreamAvailability: return self._availability_strategy.check_availability(self._logger) @property From 333d4823460706c500a07e568228664bee9c496f Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 16:42:21 -0500 Subject: [PATCH 56/74] fix tests --- .../concurrent_source/test_async_concurrent_scenarios.py | 5 ++--- .../sources/file_based/test_file_based_scenarios.py | 1 - .../python/unit_tests/sources/scenario_based/helpers.py | 1 - .../concurrent/scenarios/test_concurrent_scenarios.py | 1 - 4 files changed, 2 insertions(+), 6 deletions(-) diff --git a/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py index a98d088a5b61..bc7b376b5556 100644 --- a/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py @@ -5,7 +5,6 @@ from pathlib import PosixPath import pytest -from _pytest.capture import CaptureFixture from freezegun import freeze_time from pytest import LogCaptureFixture from unit_tests.sources.scenario_based.helpers import verify_discover, verify_read @@ -48,8 +47,8 @@ @pytest.mark.parametrize("scenario", scenarios, ids=[s.name for s in scenarios]) @freeze_time("2023-06-09T00:00:00Z") -def test_concurrent_read(caplog: LogCaptureFixture, tmp_path: PosixPath, scenario: TestScenario) -> None: - verify_read(caplog, tmp_path, scenario) +def test_concurrent_read(scenario: TestScenario) -> None: + verify_read(scenario) @pytest.mark.parametrize("scenario", scenarios, ids=[s.name for s in scenarios]) diff --git a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py index 2281c1c8c05b..f416e6a3190b 100644 --- a/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/file_based/test_file_based_scenarios.py @@ -5,7 +5,6 @@ from pathlib import PosixPath import pytest -from _pytest.capture import CaptureFixture from airbyte_cdk.sources.abstract_source import AbstractSource from freezegun import freeze_time from unit_tests.sources.file_based.scenarios.avro_scenarios import ( diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py index 7d38ef1d5105..5c664ad01292 100644 --- a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py +++ b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py @@ -10,7 +10,6 @@ import pytest from _pytest.reports import ExceptionInfo from airbyte_cdk.entrypoint import get_source_iter -from airbyte_cdk.logger import AirbyteLogFormatter from airbyte_cdk.models import AirbyteAnalyticsTraceMessage, SyncMode from airbyte_cdk.sources import AbstractSource from airbyte_cdk.test.entrypoint_wrapper import EntrypointOutput diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py index 9d24ce9e5288..db6d9b5efb9a 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/test_concurrent_scenarios.py @@ -5,7 +5,6 @@ from pathlib import PosixPath import pytest -from _pytest.capture import CaptureFixture from freezegun import freeze_time from unit_tests.sources.scenario_based.scenario_builder import TestScenario from unit_tests.sources.scenario_based.helpers import verify_discover, verify_read From 5b4d658704f0372de2702e596496bd80d35ceac8 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Tue, 2 Jan 2024 19:30:09 -0500 Subject: [PATCH 57/74] Update Salesforce --- .../connectors/source-salesforce/source_salesforce/source.py | 5 +---- .../source-salesforce/source_salesforce/streams.py | 2 +- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index e8b2c5ffe62f..f9e19f0d95bc 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -53,11 +53,8 @@ def __init__(self, catalog: Optional[ConfiguredAirbyteCatalog], config: Optional else: concurrency_level = _DEFAULT_CONCURRENCY logger.info(f"Using concurrent cdk with concurrency level {concurrency_level}") - concurrent_source = ConcurrentSource.create( - concurrency_level, concurrency_level // 2, logger, self._slice_logger, self.message_repository - ) - super().__init__(concurrent_source) self.catalog = catalog + super().__init__() @staticmethod def _get_sf_object(config: Mapping[str, Any]) -> Salesforce: diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 800f60f3aaf5..ddb705775abe 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -317,7 +317,7 @@ async def _fetch_next_page_for_chunk( return request, response -class BatchedSubStream(HttpSubStream): +class BatchedSubStream(AsyncHttpSubStream): SLICE_BATCH_SIZE = 200 def stream_slices( From 81b7e7ea567d54f5f28a056d3a2cd3b5ffd0f0be Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 5 Jan 2024 22:05:05 -0500 Subject: [PATCH 58/74] Fix Salesforce --- .../sources/abstract_source_async.py | 66 +++++----- .../http/availability_strategy_async.py | 4 +- .../sources/streams/http/http_async.py | 43 +++++-- .../streams/utils/stream_helper_async.py | 3 +- airbyte-cdk/python/setup.py | 2 + .../sources/streams/http/test_http_async.py | 2 +- .../integration_tests/configured_catalog.json | 121 +++++++++++++++++- .../connectors/source-salesforce/main.py | 2 +- .../connectors/source-salesforce/setup.py | 2 +- .../source_salesforce/api.py | 2 +- .../source_salesforce/source.py | 9 +- .../source_salesforce/streams.py | 36 +++--- 12 files changed, 225 insertions(+), 67 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 88dce2a344f7..6b0169ad8c6e 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -65,28 +65,35 @@ def _start_reader_thread(self): self.error = (e, traceback.format_exc()) def __next__(self): - with self.error_lock: - if self.error: - exception, traceback_str = self.error - self.logger.error(f"An error occurred in the async thread: {traceback_str}") - raise exception - item = self.queue.get(timeout=DEFAULT_TIMEOUT) - if isinstance(item, Sentinel): - # Sessions can only be closed once items in the stream have all been dequeued - if session := self.sessions.pop(item.name, None): - loop = asyncio.get_event_loop() - loop.create_task(session.close()) - try: - self.sentinels.pop(item.name) - except KeyError: - raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") - if not self.sentinels: - self.thread.join() - raise StopIteration + loop = asyncio.get_event_loop() + try: + with self.error_lock: + if self.error: + exception, traceback_str = self.error + self.logger.error(f"An error occurred in the async thread: {traceback_str}") + raise exception + item = self.queue.get(timeout=DEFAULT_TIMEOUT) + if isinstance(item, Sentinel): + # Sessions can only be closed once items in the stream have all been dequeued + if session := self.sessions.pop(item.name, None): + loop.create_task(session.close()) # TODO: this can be done better + try: + self.sentinels.pop(item.name) + except KeyError: + raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") + if not self.sentinels: + self.thread.join() + raise StopIteration + else: + return self.__next__() else: - return self.__next__() - else: - return item + return item + finally: + loop.create_task(self.cleanup()) + + async def cleanup(self): + for session in self.sessions.values(): + await session.close() class AsyncAbstractSource(AbstractSource, ABC): @@ -151,7 +158,7 @@ def read( n_records += 1 yield record - # print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") + print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") logger.info(f"Finished syncing {self.name}") def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): @@ -177,9 +184,10 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance pending_tasks = set() n_started, n_streams = 0, len(catalog.streams) streams_iterator = iter(catalog.streams) + exceptions = [] - while pending_tasks or n_started < n_streams: - while len(pending_tasks) <= self.session_limit and (configured_stream := next(streams_iterator, None)): + while (pending_tasks or n_started < n_streams) and not exceptions: + while len(pending_tasks) < self.session_limit and (configured_stream := next(streams_iterator, None)): if configured_stream is None: break # stream_instance = stream_instances.get("Account") # TODO @@ -194,8 +202,11 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance for task in done: if task.exception(): for remaining_task in pending_tasks: - remaining_task.cancel() - raise task.exception() + await remaining_task.cancel() + exceptions.append(task.exception()) + + if exceptions: + pass async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): try: @@ -226,7 +237,6 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) with self.reader.error_lock: self.reader.error = (e, traceback.format_exc()) - raise e except Exception as e: for message in self._emit_queued_messages(): self.queue.put(message) @@ -238,10 +248,8 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s exc = AirbyteTracedException.from_exception(e, message=display_message) with self.reader.error_lock: self.reader.error = (exc, traceback.format_exc()) - raise exc from e with self.reader.error_lock: self.reader.error = (e, traceback.format_exc()) - raise e finally: timer.finish_event() logger.info(f"Finished syncing {configured_stream.stream.name}") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py index 93c492820a0f..e647c63cf9b2 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py @@ -47,8 +47,8 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, return is_available, reason try: - await get_first_record_for_slice(stream, stream_slice) - return True, None + async for _ in get_first_record_for_slice(stream, stream_slice): + return True, None except StopIteration: logger.info(f"Successfully connected to stream {stream.name}, but got 0 records.") return True, None diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 3ecadfb73f45..3c0dfccefd20 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -173,7 +173,7 @@ def _create_aiohttp_client_request( data: Optional[Union[str, Mapping[str, Any]]] = None, ) -> aiohttp.ClientRequest: str_url = self._join_url(self.url_base, path) - str_url = "http://localhost:8000" # TODO + # str_url = "http://localhost:8000" # TODO url = URL(str_url) if self.must_deduplicate_query_params(): query_params = self.deduplicate_query_params(str_url, params) @@ -240,11 +240,7 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st raise DefaultBackoffException(response=response, error_message=error_message) elif self.raise_on_http_errors: # Raise any HTTP exceptions that happened in case there were unexpected ones - try: - response.raise_for_status() - except aiohttp.ClientResponseError as exc: - self.logger.error(response.text) - raise exc + return await self.handle_response_with_error(response) return response async def ensure_session(self) -> aiohttp.ClientSession: @@ -325,8 +321,13 @@ def _try_get_error(value: Optional[JsonType]) -> Optional[str]: return None try: - body = await response.json() - return _try_get_error(body) + if hasattr(response, "_response_json"): + return response._response_json + try: + body = await response.json() + return _try_get_error(body) + except AttributeError: + pass except json.JSONDecodeError: return None @@ -404,6 +405,31 @@ async def _fetch_next_page( response = await self._send_request(request, request_kwargs) return request, response + async def handle_response_with_error(self, response: aiohttp.ClientResponse) -> aiohttp.ClientResponse: + """ + If the response has a non-ok status code, raise an exception, otherwise return the response. + + When raising an exception, attach response json data to exception object. + """ + if response.ok: + return response + try: + error_json = await response.json() + except (json.JSONDecodeError, aiohttp.ContentTypeError): + error_json = {} + + exc = aiohttp.ClientResponseError( + response.request_info, + response.history, + status=response.status, + message=response.reason, + headers=response.headers, + ) + exc._error_json = error_json # https://github.com/aio-libs/aiohttp/issues/3248 + text = await response.text() + self.logger.error(text) + raise exc + class AsyncHttpSubStream(AsyncHttpStream, ABC): def __init__(self, parent: AsyncHttpStream, **kwargs: Any): @@ -416,6 +442,7 @@ def __init__(self, parent: AsyncHttpStream, **kwargs: Any): async def stream_slices( self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None ) -> Iterable[Optional[Mapping[str, Any]]]: + await self.parent.ensure_session() # iterate over all parent stream_slices async for stream_slice in self.parent.stream_slices( sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_state=stream_state diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py index efdb50922a59..bb16e3418f7b 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py @@ -32,5 +32,4 @@ async def get_first_record_for_slice(stream: AsyncStream, stream_slice: Optional # We wrap the return output of read_records() because some implementations return types that are iterable, # but not iterators such as lists or tuples async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slice): - return record - raise StopIteration(f"No records in stream {stream.name}") + yield record diff --git a/airbyte-cdk/python/setup.py b/airbyte-cdk/python/setup.py index 61aab4a298cd..646633e2713e 100644 --- a/airbyte-cdk/python/setup.py +++ b/airbyte-cdk/python/setup.py @@ -65,6 +65,7 @@ packages=find_packages(exclude=("unit_tests",)), package_data={"airbyte_cdk": ["py.typed", "sources/declarative/declarative_component_schema.yaml"]}, install_requires=[ + "aiohttp~=3.8.6" "aiohttp-client-cache[aiosqlite]", "aiosqlite", "airbyte-protocol-models==0.5.1", @@ -94,6 +95,7 @@ "freezegun", "mypy", "pytest", + "pytest-asyncio", "pytest-cov", "pytest-mock", "requests-mock", diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index 6389ff252e93..29345b94bccb 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -68,7 +68,7 @@ def test_request_kwargs_used(mocker): m.assert_any_call(stream.url_base, "GET", **request_kwargs) m.assert_called_once() - loop.run_until_complete(stream._session.close()) + loop.run_until_complete(stream._session.close()) # TODO - find a way to not manually close after each test async def read_records(stream, sync_mode=SyncMode.full_refresh, stream_slice=None): diff --git a/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json b/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json index 53f4ded6de8f..1d82d43b11a1 100644 --- a/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json +++ b/airbyte-integrations/connectors/source-salesforce/integration_tests/configured_catalog.json @@ -1 +1,120 @@ -{"streams": [{"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}, {"stream": {"name": "Account", "json_schema": {}, "supported_sync_modes": ["full_refresh", "incremental"], "source_defined_cursor": true, "default_cursor_field": ["SystemModstamp"], "source_defined_primary_key": [["Id"]]}, "sync_mode": "full_refresh", "destination_sync_mode": "append"}]} \ No newline at end of file +{ + "streams": [ + { + "stream": { + "name": "Account", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "ActiveFeatureLicenseMetric", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "ActivePermSetLicenseMetric", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "ActiveProfileMetric", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "AppDefinition", + "json_schema": {}, + "supported_sync_modes": ["full_refresh"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "overwrite" + }, + { + "stream": { + "name": "Asset", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "FormulaFunctionAllowedType", + "json_schema": {}, + "supported_sync_modes": ["full_refresh"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "overwrite" + }, + { + "stream": { + "name": "ObjectPermissions", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "PermissionSetTabSetting", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["SystemModstamp"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + }, + { + "stream": { + "name": "LeadHistory", + "json_schema": {}, + "supported_sync_modes": ["full_refresh", "incremental"], + "source_defined_cursor": true, + "default_cursor_field": ["CreatedDate"], + "source_defined_primary_key": [["Id"]] + }, + "sync_mode": "full_refresh", + "destination_sync_mode": "append" + } + ] +} diff --git a/airbyte-integrations/connectors/source-salesforce/main.py b/airbyte-integrations/connectors/source-salesforce/main.py index 5ec9f05e1042..d871182bd494 100644 --- a/airbyte-integrations/connectors/source-salesforce/main.py +++ b/airbyte-integrations/connectors/source-salesforce/main.py @@ -39,7 +39,7 @@ def _get_source(args: List[str]): if __name__ == "__main__": - _args = sys.argv[1:] + _args = ['read', '--config', 'secrets/config.json', '--catalog', 'integration_tests/configured_catalog.json'] source = _get_source(_args) if source: launch(source, _args) diff --git a/airbyte-integrations/connectors/source-salesforce/setup.py b/airbyte-integrations/connectors/source-salesforce/setup.py index 22e6250d4660..f1b4ddfaedd8 100644 --- a/airbyte-integrations/connectors/source-salesforce/setup.py +++ b/airbyte-integrations/connectors/source-salesforce/setup.py @@ -7,7 +7,7 @@ MAIN_REQUIREMENTS = ["airbyte-cdk~=0.55.2", "pandas"] -TEST_REQUIREMENTS = ["freezegun", "pytest~=6.1", "pytest-mock~=3.6", "requests-mock~=1.9.3", "pytest-timeout"] +TEST_REQUIREMENTS = ["aioresponses", "freezegun", "pytest~=6.1", "pytest-asyncio", "pytest-mock~=3.6", "requests-mock~=1.9.3", "pytest-timeout"] setup( name="source_salesforce", diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py index 92bd614c7a2d..eb0eed9ef70d 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py @@ -283,7 +283,7 @@ def get_validated_streams(self, config: Mapping[str, Any], catalog: ConfiguredAi if catalog: return { - configured_stream.stream.name: stream_objects["Account"] + configured_stream.stream.name: stream_objects[configured_stream.stream.name] for configured_stream in catalog.streams if configured_stream.stream.name in stream_objects } diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index f9e19f0d95bc..9b0d9b3e4926 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -6,6 +6,7 @@ from datetime import datetime from typing import Any, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union +import aiohttp import requests from airbyte_cdk import AirbyteLogger from airbyte_cdk.logger import AirbyteLogFormatter @@ -224,11 +225,11 @@ async def _read_stream( try: async for record in super()._read_stream(logger, stream_instance, configured_stream, state_manager, internal_config): yield record - except exceptions.HTTPError as error: - error_data = error.response.json()[0] + except aiohttp.ClientResponseError as error: + error_data = error._error_json if hasattr(error, "_error_json") else {} error_code = error_data.get("errorCode") - url = error.response.url - if error.response.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + url = error.request_info.url + if error.status == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": logger.warning(f"API Call {url} limit is exceeded. Error message: '{error_data.get('message')}'") raise AirbyteStopSync() # if got 403 rate limit response, finish the sync with success. raise error diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index ddb705775abe..087a3fcd0965 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -108,7 +108,7 @@ def get_json_schema(self) -> Mapping[str, Any]: return self.schema async def get_error_display_message(self, exception: BaseException) -> Optional[str]: - if isinstance(exception, exceptions.ConnectionError): + if isinstance(exception, aiohttp.ClientResponseError): return f"After {self.max_retries} retries the connector has failed with a network error. It looks like Salesforce API experienced temporary instability, please try again later." return await super().get_error_display_message(exception) @@ -268,7 +268,7 @@ async def f(): continue # stick together different parts of records by their primary key and emit if a record is complete - for record in chunk_page_records: + async for record in records_generator_fn(request, response, stream_state, stream_slice): property_chunk.record_counter += 1 record_id = record[self.primary_key] if record_id not in records_by_primary_key: @@ -320,15 +320,16 @@ async def _fetch_next_page_for_chunk( class BatchedSubStream(AsyncHttpSubStream): SLICE_BATCH_SIZE = 200 - def stream_slices( + async def stream_slices( self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None ) -> Iterable[Optional[Mapping[str, Any]]]: """Instead of yielding one parent record at a time, make stream slice contain a batch of parent records. It allows to get records by one requests (instead of only one). """ + await self.ensure_session() # TODO: should this be self or super? batched_slice = [] - for stream_slice in super().stream_slices(sync_mode, cursor_field, stream_state): + async for stream_slice in super().stream_slices(sync_mode, cursor_field, stream_state): if len(batched_slice) == self.SLICE_BATCH_SIZE: yield {"parents": batched_slice} batched_slice = [] @@ -352,13 +353,12 @@ def path(self, next_page_token: Mapping[str, Any] = None, **kwargs: Any) -> str: transformer = TypeTransformer(TransformConfig.CustomSchemaNormalization | TransformConfig.DefaultSchemaNormalization) @default_backoff_handler(max_tries=5, factor=15) - async def _send_http_request(self, method: str, url: str, json: dict = None, headers: dict = None, stream=False) -> aiohttp.ClientResponse: + async def _send_http_request(self, method: str, url: str, json: dict = None, headers: dict = None, stream=False) -> aiohttp.ClientResponse: # TODO: how to handle the stream argument headers = self.authenticator.get_auth_header() if not headers else headers | self.authenticator.get_auth_header() response = await self._session.request(method, url=url, headers=headers, json=json) if response.status not in [200, 204]: self.logger.error(f"error body: {await response.text()}, sobject options: {self.sobject_options}") - response.raise_for_status() - return response # TODO: how to handle the stream argument + return await self.handle_response_with_error(response) async def create_stream_job(self, query: str, url: str) -> Optional[str]: """ @@ -369,8 +369,8 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: response = await self._send_http_request("POST", url, json=json) job_id: str = (await response.json())["id"] return job_id - except exceptions.HTTPError as error: # TODO: which errors? - if error.response.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: + except aiohttp.ClientResponseError as error: # TODO: which errors? + if error.status in [codes.FORBIDDEN, codes.BAD_REQUEST]: # A part of streams can't be used by BULK API. Every API version can have a custom list of # these sobjects. Another part of them can be generated dynamically. That's why we can't track # them preliminarily and there is only one way is to except error with necessary messages about @@ -383,7 +383,9 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: # updated query: "Select Name, (Select Subject,ActivityType from ActivityHistories) from Contact" # The second variant forces customisation for every case (ActivityHistory, ActivityHistories etc). # And the main problem is these subqueries doesn't support CSV response format. - error_data = error.response.json()[0] + if not hasattr(error, "_error_json"): + raise NotImplementedError("!!!!!!!!!!!!! this didn't use `handle_response_with_error`") + error_data = error._error_json or {} error_code = error_data.get("errorCode") error_message = error_data.get("message", "") if error_message == "Selecting compound data not supported in Bulk Query" or ( @@ -393,29 +395,29 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: f"Cannot receive data for stream '{self.name}' using BULK API, " f"sobject options: {self.sobject_options}, error message: '{error_message}'" ) - elif error.response.status_code == codes.FORBIDDEN and error_code != "REQUEST_LIMIT_EXCEEDED": + elif error.status == codes.FORBIDDEN and error_code != "REQUEST_LIMIT_EXCEEDED": self.logger.error( f"Cannot receive data for stream '{self.name}' ," f"sobject options: {self.sobject_options}, error message: '{error_message}'" ) - elif error.response.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + elif error.status == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": self.logger.error( f"Cannot receive data for stream '{self.name}' ," f"sobject options: {self.sobject_options}, Error message: '{error_data.get('message')}'" ) - elif error.response.status_code == codes.BAD_REQUEST and error_message.endswith("does not support query"): + elif error.status == codes.BAD_REQUEST and error_message.endswith("does not support query"): self.logger.error( f"The stream '{self.name}' is not queryable, " f"sobject options: {self.sobject_options}, error message: '{error_message}'" ) elif ( - error.response.status_code == codes.BAD_REQUEST + error.status == codes.BAD_REQUEST and error_code == "API_ERROR" and error_message.startswith("Implementation restriction") ): message = f"Unable to sync '{self.name}'. To prevent future syncs from failing, ensure the authenticated user has \"View all Data\" permissions." raise AirbyteTracedException(message=message, failure_type=FailureType.config_error, exception=error) - elif error.response.status_code == codes.BAD_REQUEST and error_code == "LIMIT_EXCEEDED": + elif error.status == codes.BAD_REQUEST and error_code == "LIMIT_EXCEEDED": message = "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed." self.logger.error(message) else: @@ -436,8 +438,8 @@ async def wait_for_job(self, url: str) -> str: while pendulum.now() < expiration_time: try: job_info = await (await self._send_http_request("GET", url=url)).json() - except exceptions.HTTPError as error: - error_data = error.response.json()[0] + except aiohttp.ClientResponseError as error: + error_data = error._error_json error_code = error_data.get("errorCode") error_message = error_data.get("message", "") if ( From 3741cc90e0d51a9c64eb25374f08787ef006c00f Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 5 Jan 2024 22:15:02 -0500 Subject: [PATCH 59/74] Salesforce tests --- .../source-salesforce/unit_tests/api_test.py | 743 ++++++++++-------- .../unit_tests/test_memory.py | 17 +- 2 files changed, 415 insertions(+), 345 deletions(-) diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index 8f87e2bd58cd..cd24b089b100 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -10,17 +10,20 @@ from datetime import datetime from typing import List from unittest.mock import Mock +from yarl import URL +import aiohttp import freezegun import pendulum import pytest import requests_mock +from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import AirbyteStream, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, DestinationSyncMode, SyncMode, Type +from airbyte_cdk.sources import abstract_source_async from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade from airbyte_cdk.utils import AirbyteTracedException from conftest import encoding_symbols_parameters, generate_stream -from requests.exceptions import HTTPError from source_salesforce.api import Salesforce from source_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING from source_salesforce.source import SourceSalesforce @@ -29,10 +32,8 @@ BulkIncrementalSalesforceStream, BulkSalesforceStream, BulkSalesforceSubStream, - Describe, IncrementalRestSalesforceStream, RestSalesforceStream, - SalesforceStream, ) _ANY_CATALOG = ConfiguredAirbyteCatalog.parse_obj({"streams": []}) @@ -81,39 +82,55 @@ def test_login_authentication_error_handler( assert msg == expected_error_msg -def test_bulk_sync_creation_failed(stream_config, stream_api): +@pytest.mark.asyncio +async def test_bulk_sync_creation_failed(stream_config, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with requests_mock.Mocker() as m: - m.register_uri("POST", stream.path(), status_code=400, json=[{"message": "test_error"}]) - with pytest.raises(HTTPError) as err: - stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) - next(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)) - assert err.value.response.json()[0]["message"] == "test_error" + await stream.ensure_session() + + def callback(*args, **kwargs): + return CallbackResult(status=400, reason="test_error", content_type="application/json") + + with aioresponses() as m: + m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) + with pytest.raises(aiohttp.ClientResponseError) as err: + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + assert err.value.message == "test_error" + await stream._session.close() -def test_bulk_stream_fallback_to_rest(mocker, requests_mock, stream_config, stream_api): +@pytest.mark.asyncio +async def test_bulk_stream_fallback_to_rest(stream_config, stream_api): """ Here we mock BULK API with response returning error, saying BULK is not supported for this kind of entity. On the other hand, we mock REST API for this same entity with a successful response. After having instantiated a BulkStream, sync should succeed in case it falls back to REST API. Otherwise it would throw an error. """ stream = generate_stream("CustomEntity", stream_config, stream_api) - # mock a BULK API - requests_mock.register_uri( - "POST", - "https://fase-account.salesforce.com/services/data/v57.0/jobs/query", - status_code=400, - json=[{"errorCode": "INVALIDENTITY", "message": "CustomEntity is not supported by the Bulk API"}], - ) + await stream.ensure_session() + + def callback(*args, **kwargs): + return CallbackResult(status=400, payload={"errorCode": "INVALIDENTITY", "message": "CustomEntity is not supported by the Bulk API"}, content_type="application/json") + rest_stream_records = [ {"id": 1, "name": "custom entity", "created": "2010-11-11"}, {"id": 11, "name": "custom entity", "created": "2020-01-02"}, ] - # mock REST API - mocker.patch("source_salesforce.source.RestSalesforceStream.read_records", lambda *args, **kwargs: iter(rest_stream_records)) - assert type(stream) is BulkIncrementalSalesforceStream - stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) - assert list(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)) == rest_stream_records + async def get_records(*args, **kwargs): + nonlocal rest_stream_records + for record in rest_stream_records: + yield record + + with aioresponses() as m: + # mock a BULK API + m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) + # mock REST API + stream.read_records = get_records + assert type(stream) is BulkIncrementalSalesforceStream + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + assert [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] == rest_stream_records + + await stream._session.close() def test_stream_unsupported_by_bulk(stream_config, stream_api): @@ -135,101 +152,133 @@ def test_stream_contains_unsupported_properties_by_bulk(stream_config, stream_ap assert not isinstance(stream, BulkSalesforceStream) -def test_bulk_sync_pagination(stream_config, stream_api, requests_mock): +@pytest.mark.asyncio +async def test_bulk_sync_pagination(stream_config, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() job_id = "fake_job" - requests_mock.register_uri("POST", stream.path(), json={"id": job_id}) - requests_mock.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "JobComplete"}) - resp_text = ["Field1,LastModifiedDate,ID"] + [f"test,2021-11-16,{i}" for i in range(5)] - result_uri = requests_mock.register_uri( - "GET", - stream.path() + f"/{job_id}/results", - [ - {"text": "\n".join(resp_text), "headers": {"Sforce-Locator": "somelocator_1"}}, - {"text": "\n".join(resp_text), "headers": {"Sforce-Locator": "somelocator_2"}}, - {"text": "\n".join(resp_text), "headers": {"Sforce-Locator": "null"}}, - ], - ) - requests_mock.register_uri("DELETE", stream.path() + f"/{job_id}") + call_counter = 0 + + def cb1(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + return CallbackResult(headers={"Sforce-Locator": "somelocator_1"}, body="\n".join(resp_text)) + + def cb2(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + return CallbackResult(headers={"Sforce-Locator": "somelocator_2"}, body="\n".join(resp_text)) - stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) - loaded_ids = [int(record["ID"]) for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] - assert loaded_ids == [0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4] - assert result_uri.call_count == 3 - assert result_uri.request_history[1].query == "locator=somelocator_1" - assert result_uri.request_history[2].query == "locator=somelocator_2" + def cb3(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + return CallbackResult(headers={"Sforce-Locator": "null"}, body="\n".join(resp_text)) + with aioresponses() as m: + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + m.post(f"{base_url}", callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.get(f"{base_url}/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + resp_text = ["Field1,LastModifiedDate,ID"] + [f"test,2021-11-16,{i}" for i in range(5)] + m.get(f"{base_url}/{job_id}/results", callback=cb1) + m.get(f"{base_url}/{job_id}/results?locator=somelocator_1", callback=cb2) + m.get(f"{base_url}/{job_id}/results?locator=somelocator_2", callback=cb3) + m.delete(base_url + f"/{job_id}") + + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + loaded_ids = [int(record["ID"]) async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + assert loaded_ids == [0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4] + assert call_counter == 3 + await stream._session.close() def _prepare_mock(m, stream): job_id = "fake_job_1" - m.register_uri("POST", stream.path(), json={"id": job_id}) - m.register_uri("DELETE", stream.path() + f"/{job_id}") - m.register_uri("GET", stream.path() + f"/{job_id}/results", text="Field1,LastModifiedDate,ID\ntest,2021-11-16,1") - m.register_uri("PATCH", stream.path() + f"/{job_id}", text="") + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.delete(base_url + f"/{job_id}") + m.get(base_url + f"/{job_id}/results", callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2021-11-16,1")) + m.patch(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(body="")) return job_id -def _get_result_id(stream): - stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) - return int(list(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices))[0]["ID"]) +async def _get_result_id(stream): + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + records = [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + return int(list(records)[0]["ID"]) -def test_bulk_sync_successful(stream_config, stream_api): +@pytest.mark.asyncio +async def test_bulk_sync_successful(stream_config, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with requests_mock.Mocker() as m: + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + + with aioresponses() as m: job_id = _prepare_mock(m, stream) - m.register_uri("GET", stream.path() + f"/{job_id}", [{"json": {"state": "JobComplete"}}]) - assert _get_result_id(stream) == 1 + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + assert await _get_result_id(stream) == 1 -def test_bulk_sync_successful_long_response(stream_config, stream_api): +@pytest.mark.asyncio +async def test_bulk_sync_successful_long_response(stream_config, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with requests_mock.Mocker() as m: + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: job_id = _prepare_mock(m, stream) - m.register_uri( - "GET", - stream.path() + f"/{job_id}", - [ - {"json": {"state": "UploadComplete", "id": job_id}}, - {"json": {"state": "InProgress", "id": job_id}}, - {"json": {"state": "JobComplete", "id": job_id}}, - ], - ) - assert _get_result_id(stream) == 1 + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "UploadComplete", "id": job_id})) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete", "id": job_id})) + assert await _get_result_id(stream) == 1 # maximum timeout is wait_timeout * max_retry_attempt # this test tries to check a job state 17 times with +-1second for very one +@pytest.mark.asyncio @pytest.mark.timeout(17) -def test_bulk_sync_successful_retry(stream_config, stream_api): +async def test_bulk_sync_successful_retry(stream_config, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" - with requests_mock.Mocker() as m: + with aioresponses() as m: job_id = _prepare_mock(m, stream) # 2 failed attempts, 3rd one should be successful states = [{"json": {"state": "InProgress", "id": job_id}}] * 17 states.append({"json": {"state": "JobComplete", "id": job_id}}) # raise Exception(states) - m.register_uri("GET", stream.path() + f"/{job_id}", states) - assert _get_result_id(stream) == 1 + for _ in range(17): + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete", "id": job_id})) + + assert await _get_result_id(stream) == 1 +@pytest.mark.asyncio @pytest.mark.timeout(30) -def test_bulk_sync_failed_retry(stream_config, stream_api): +async def test_bulk_sync_failed_retry(stream_config, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds - with requests_mock.Mocker() as m: + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: job_id = _prepare_mock(m, stream) - m.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "InProgress", "id": job_id}) + m.get(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) + m.post(base_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) with pytest.raises(Exception) as err: - stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) - next(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)) + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] assert "stream using BULK API was failed" in str(err.value) + await stream._session.close() @pytest.mark.parametrize( "start_date_provided,stream_name,expected_start_date", @@ -264,67 +313,77 @@ def test_stream_start_datetime_format_should_not_changed(stream_config, stream_a assert stream.start_date == "2010-01-18T21:18:20Z" -def test_download_data_filter_null_bytes(stream_config, stream_api): +@pytest.mark.asyncio +async def test_download_data_filter_null_bytes(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() - with requests_mock.Mocker() as m: - m.register_uri("GET", job_full_url_results, content=b"\x00") - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + with aioresponses() as m: + m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b"\x00")) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [] - m.register_uri("GET", job_full_url_results, content=b'"Id","IsDeleted"\n\x00"0014W000027f6UwQAI","false"\n\x00\x00') - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b'"Id","IsDeleted"\n\x00"0014W000027f6UwQAI","false"\n\x00\x00')) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"Id": "0014W000027f6UwQAI", "IsDeleted": "false"}] -def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): +@pytest.mark.asyncio +async def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() - with requests_mock.Mocker() as m: - m.register_uri("GET", job_full_url_results, content=b'"IsDeleted","Age"\n"false",24\n') - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + with aioresponses() as m: + m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b'"IsDeleted","Age"\n"false",24\n')) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"IsDeleted": "false", "Age": "24"}] -def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): +@pytest.mark.asyncio +async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() - with requests_mock.Mocker() as m: - m.register_uri("GET", job_full_url_results, content=b'"ZipCode"\n"01234"\n') - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + with aioresponses() as m: + m.get(job_full_url_results, body=b'"ZipCode"\n"01234"\n') + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"ZipCode": "01234"}] -def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): +@pytest.mark.asyncio +async def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() - with requests_mock.Mocker() as m: - m.register_uri("GET", job_full_url_results, content=b'"IsDeleted","Age","Name"\n"false",,"Airbyte"\n') - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + with aioresponses() as m: + m.get(job_full_url_results, body=b'"IsDeleted","Age","Name"\n"false",,"Airbyte"\n') + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"IsDeleted": "false", "Age": None, "Name": "Airbyte"}] +@pytest.mark.asyncio @pytest.mark.parametrize( "chunk_size, content_type_header, content, expected_result", encoding_symbols_parameters(), ids=[f"charset: {x[1]}, chunk_size: {x[0]}" for x in encoding_symbols_parameters()], ) -def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): +async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() - with requests_mock.Mocker() as m: - m.register_uri("GET", job_full_url_results, headers=content_type_header, content=content) - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + with aioresponses() as m: + m.get(job_full_url_results, headers=content_type_header, body=content) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == expected_result @@ -342,7 +401,7 @@ def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_he ), ), ) -def test_check_connection_rate_limit( +async def test_check_connection_rate_limit( stream_config, login_status_code, login_json_resp, discovery_status_code, discovery_resp_json, expected_error_msg ): source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) @@ -366,13 +425,15 @@ def configure_request_params_mock(stream_1, stream_2): stream_2.request_params.return_value = {"q": "query"} -def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): +@pytest.mark.asyncio +async def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): """ Connector should stop the sync if one stream reached rate limit stream_1, stream_2, stream_3, ... While reading `stream_1` if 403 (Rate Limit) is received, it should finish that stream with success and stop the sync process. Next streams should not be executed. """ + abstract_source_async.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time stream_config.update({"start_date": "2021-10-01"}) stream_1: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) stream_2: BulkIncrementalSalesforceStream = generate_stream("Asset", stream_config, stream_api) @@ -387,54 +448,63 @@ def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): source.streams.return_value = streams logger = logging.getLogger("airbyte") - json_response = [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."}] - with requests_mock.Mocker() as m: - for stream in streams: + json_response = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} + + orig_read_stream = source._read_stream + + async def patched_read_stream(*args, **kwargs): + base_url = f"{stream_1.sf_api.instance_url}{stream_1.path()}" + with aioresponses() as m: creation_responses = [] for page in [1, 2]: - job_id = f"fake_job_{page}_{stream.name}" - creation_responses.append({"json": {"id": job_id}}) + job_id = f"fake_job_{page}_{stream_1.name}" + creation_responses.append({"id": job_id}) - m.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "JobComplete"}) + m.get(base_url + f"/{job_id}", callback=lambda *_, **__: CallbackResult(payload={"state": "JobComplete"})) resp = ["Field1,LastModifiedDate,Id"] + [f"test,2021-10-0{i},{i}" for i in range(1, 7)] # 6 records per page if page == 1: # Read the first page successfully - m.register_uri("GET", stream.path() + f"/{job_id}/results", text="\n".join(resp)) + m.get(base_url + f"/{job_id}/results", callback=lambda *_, **__: CallbackResult(body="\n".join(resp))) else: # Requesting for results when reading second page should fail with 403 (Rate Limit error) - m.register_uri("GET", stream.path() + f"/{job_id}/results", status_code=403, json=json_response) + m.get(base_url + f"/{job_id}/results", status=403, callback=lambda *_, **__: CallbackResult(status=403, payload=json_response)) - m.register_uri("DELETE", stream.path() + f"/{job_id}") + m.delete(base_url + f"/{job_id}") - m.register_uri("POST", stream.path(), creation_responses) + def cb(response): + return lambda *_, **__: CallbackResult(payload=response) - result = [i for i in source.read(logger=logger, config=stream_config, catalog=bulk_catalog, state=state)] - assert stream_1.request_params.called - assert ( - not stream_2.request_params.called - ), "The second stream should not be executed, because the first stream finished with Rate Limit." + for response in creation_responses: + m.post(base_url, callback=cb(response)) - records = [item for item in result if item.type == Type.RECORD] - assert len(records) == 6 # stream page size: 6 + async for r in orig_read_stream(**kwargs): + yield r - state_record = [item for item in result if item.type == Type.STATE][0] - assert state_record.state.data["Account"]["LastModifiedDate"] == "2021-10-05T00:00:00+00:00" # state checkpoint interval is 5. + source._read_stream = patched_read_stream + result = [i for i in source.read(logger=logger, config=stream_config, catalog=bulk_catalog, state=state)] + assert stream_1.request_params.called + assert ( + not stream_2.request_params.called + ), "The second stream should not be executed, because the first stream finished with Rate Limit." -def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): - """ - Connector should stop the sync if one stream reached rate limit - stream_1, stream_2, stream_3, ... - While reading `stream_1` if 403 (Rate Limit) is received, it should finish that stream with success and stop the sync process. - Next streams should not be executed. - """ - stream_config.update({"start_date": "2021-11-01"}) + records = [item for item in result if item.type == Type.RECORD] + assert len(records) == 6 # stream page size: 6 + + state_record = [item for item in result if item.type == Type.STATE][0] + assert state_record.state.data["Account"]["LastModifiedDate"] == "2021-10-05T00:00:00+00:00" # state checkpoint interval is 5. + await stream_1._session.close() + await stream_2._session.close() + +@pytest.mark.asyncio +async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): + abstract_source_async.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time + stream_config.update({"start_date": "2021-11-01"}) stream_1: IncrementalRestSalesforceStream = generate_stream("KnowledgeArticle", stream_config, stream_api) stream_2: IncrementalRestSalesforceStream = generate_stream("AcceptedEventRelation", stream_config, stream_api) - stream_1.state_checkpoint_interval = 3 configure_request_params_mock(stream_1, stream_2) @@ -472,72 +542,107 @@ def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): }, ], } - response_2 = [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."}] + response_2 = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} - with requests_mock.Mocker() as m: - m.register_uri("GET", stream_1.path(), json=response_1, status_code=200) - m.register_uri("GET", next_page_url, json=response_2, status_code=403) + def cb1(*args, **kwargs): + return CallbackResult(payload=response_1, status=200) + + def cb2(*args, **kwargs): + return CallbackResult(payload=response_2, status=403, reason="") - result = [i for i in source.read(logger=logger, config=stream_config, catalog=rest_catalog, state=state)] + orig_read_records_s1 = stream_1.read_records + orig_read_records_s2 = stream_2.read_records - assert stream_1.request_params.called - assert ( - not stream_2.request_params.called - ), "The second stream should not be executed, because the first stream finished with Rate Limit." + async def patched_read_records_s1(*args, **kwargs): + with aioresponses() as m: + m.get(re.compile(re.escape(rf"{stream_1.sf_api.instance_url}{stream_1.path()}") + rf"\??.*"), repeat=True, callback=cb1) + m.get(re.compile(re.escape(rf"{stream_1.sf_api.instance_url}{next_page_url}") + rf"\??.*"), repeat=True, callback=cb2) - records = [item for item in result if item.type == Type.RECORD] - assert len(records) == 5 + async for r in orig_read_records_s1(**kwargs): + yield r - state_record = [item for item in result if item.type == Type.STATE][0] - assert state_record.state.data["KnowledgeArticle"]["LastModifiedDate"] == "2021-11-17T00:00:00+00:00" + async def patched_read_records_s2(*args, **kwargs): + with aioresponses() as m: + m.get(re.compile(re.escape(rf"{stream_2.sf_api.instance_url}{stream_2.path()}") + rf"\??.*"), repeat=True, callback=cb1) + m.get(re.compile(re.escape(rf"{stream_2.sf_api.instance_url}{next_page_url}") + rf"\??.*"), repeat=True, callback=cb1) + async for r in orig_read_records_s2(**kwargs): + yield r + async def check_availability(*args, **kwargs): + return (True, None) -def test_pagination_rest(stream_config, stream_api): + stream_1.read_records = lambda *args, **kwargs: patched_read_records_s1(stream_1, *args, **kwargs) + stream_1.check_availability = check_availability + stream_2.read_records = lambda *args, **kwargs: patched_read_records_s2(stream_2, *args, **kwargs) + stream_2.check_availability = check_availability + + result = [i for i in source.read(logger=logger, config=stream_config, catalog=rest_catalog, state=state)] + + assert stream_1.request_params.called + assert ( + not stream_2.request_params.called + ), "The second stream should not be executed, because the first stream finished with Rate Limit." + + records = [item for item in result if item.type == Type.RECORD] + assert len(records) == 5 + + state_record = [item for item in result if item.type == Type.STATE][0] + assert state_record.state.data["KnowledgeArticle"]["LastModifiedDate"] == "2021-11-17T00:00:00+00:00" + await stream_1._session.close() + await stream_2._session.close() + + +@pytest.mark.asyncio +async def test_pagination_rest(stream_config, stream_api): stream_name = "AcceptedEventRelation" stream: RestSalesforceStream = generate_stream(stream_name, stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds next_page_url = "/services/data/v57.0/query/012345" - with requests_mock.Mocker() as m: - resp_1 = { - "done": False, - "totalSize": 4, - "nextRecordsUrl": next_page_url, - "records": [ - { - "ID": 1, - "LastModifiedDate": "2021-11-15", - }, - { - "ID": 2, - "LastModifiedDate": "2021-11-16", - }, - ], - } - resp_2 = { - "done": True, - "totalSize": 4, - "records": [ - { - "ID": 3, - "LastModifiedDate": "2021-11-17", - }, - { - "ID": 4, - "LastModifiedDate": "2021-11-18", - }, - ], - } - - m.register_uri("GET", stream.path(), json=resp_1) - m.register_uri("GET", next_page_url, json=resp_2) - - records = [record for record in stream.read_records(sync_mode=SyncMode.full_refresh)] + await stream.ensure_session() + + resp_1 = { + "done": False, + "totalSize": 4, + "nextRecordsUrl": next_page_url, + "records": [ + { + "ID": 1, + "LastModifiedDate": "2021-11-15", + }, + { + "ID": 2, + "LastModifiedDate": "2021-11-16", + }, + ], + } + resp_2 = { + "done": True, + "totalSize": 4, + "records": [ + { + "ID": 3, + "LastModifiedDate": "2021-11-17", + }, + { + "ID": 4, + "LastModifiedDate": "2021-11-18", + }, + ], + } + + with aioresponses() as m: + m.get(re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0\??.*"), callback=lambda *args, **kwargs: CallbackResult(payload=resp_1)) + m.get("https://fase-account.salesforce.com" + next_page_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload=resp_2)) + + records = [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh)] assert len(records) == 4 -def test_csv_reader_dialect_unix(): +@pytest.mark.asyncio +async def test_csv_reader_dialect_unix(): stream: BulkSalesforceStream = BulkSalesforceStream(stream_name=None, sf_api=None, pk=None) url_results = "https://fake-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + await stream.ensure_session() data = [ {"Id": "1", "Name": '"first_name" "last_name"'}, @@ -552,9 +657,9 @@ def test_csv_reader_dialect_unix(): writer.writerow(line) text = csvfile.getvalue() - with requests_mock.Mocker() as m: - m.register_uri("GET", url_results, text=text) - tmp_file, response_encoding, _ = stream.download_data(url=url_results) + with aioresponses() as m: + m.get(url_results, callback=lambda *args, **kwargs: CallbackResult(body=text)) + tmp_file, response_encoding, _ = await stream.download_data(url=url_results) result = [i for i in stream.read_with_chunks(tmp_file, response_encoding)] assert result == data @@ -576,7 +681,7 @@ def test_csv_reader_dialect_unix(): ), ), ) -def test_forwarding_sobject_options(stream_config, stream_names, catalog_stream_names) -> None: +async def test_forwarding_sobject_options(stream_config, stream_names, catalog_stream_names) -> None: sobjects_matcher = re.compile("/sobjects$") token_matcher = re.compile("/token$") describe_matcher = re.compile("/describe$") @@ -638,46 +743,6 @@ def test_forwarding_sobject_options(stream_config, stream_names, catalog_stream_ return -@pytest.mark.parametrize( - "stream_names,catalog_stream_names,", - ( - ( - ["stream_1", "stream_2", "Describe"], - None, - ), - ( - ["stream_1", "stream_2"], - ["stream_1", "stream_2", "Describe"], - ), - ( - ["stream_1", "stream_2", "stream_3", "Describe"], - ["stream_1", "Describe"], - ), - ), -) -def test_unspecified_and_incremental_streams_are_not_concurrent(stream_config, stream_names, catalog_stream_names) -> None: - for stream in _get_streams(stream_config, stream_names, catalog_stream_names, SyncMode.incremental): - assert isinstance(stream, (SalesforceStream, Describe)) - - -@pytest.mark.parametrize( - "stream_names,catalog_stream_names,", - ( - ( - ["stream_1", "stream_2"], - ["stream_1", "stream_2", "Describe"], - ), - ( - ["stream_1", "stream_2", "stream_3", "Describe"], - ["stream_1", "Describe"], - ), - ), -) -def test_full_refresh_streams_are_concurrent(stream_config, stream_names, catalog_stream_names) -> None: - for stream in _get_streams(stream_config, stream_names, catalog_stream_names, SyncMode.full_refresh): - assert isinstance(stream, StreamFacade) - - def _get_streams(stream_config, stream_names, catalog_stream_names, sync_type) -> List[Stream]: sobjects_matcher = re.compile("/sobjects$") token_matcher = re.compile("/token$") @@ -759,8 +824,10 @@ def test_rest_stream_init_with_too_many_properties(stream_config, stream_api_v2_ generate_stream("Account", stream_config, stream_api_v2_too_many_properties) -def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): +@pytest.mark.asyncio +async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): stream = generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + await stream.ensure_session() chunks = list(stream.chunk_properties()) for chunk in chunks: assert stream.primary_key in chunk @@ -768,29 +835,27 @@ def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties assert stream.too_many_properties assert stream.primary_key assert type(stream) == RestSalesforceStream - url = next_page_url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" - requests_mock.get( - url, - [ - { - "json": { + next_page_url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" + url_pattern = re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0/queryAll\??.*") + with aioresponses() as m: + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={ "records": [ {"Id": 1, "propertyA": "A"}, {"Id": 2, "propertyA": "A"}, {"Id": 3, "propertyA": "A"}, {"Id": 4, "propertyA": "A"}, ] - } - }, - {"json": {"nextRecordsUrl": next_page_url, "records": [{"Id": 1, "propertyB": "B"}, {"Id": 2, "propertyB": "B"}]}}, - # 2 for 2 chunks above - *[{"json": {"records": [{"Id": 1}, {"Id": 2}], "nextRecordsUrl": next_page_url}} for _ in range(chunks_len - 2)], - {"json": {"records": [{"Id": 3, "propertyB": "B"}, {"Id": 4, "propertyB": "B"}]}}, - # 2 for 1 chunk above and 1 chunk had no next page - *[{"json": {"records": [{"Id": 3}, {"Id": 4}]}} for _ in range(chunks_len - 2)], - ], - ) - records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) + })) + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"nextRecordsUrl": next_page_url, "records": [{"Id": 1, "propertyB": "B"}, {"Id": 2, "propertyB": "B"}]})) + # 2 for 2 chunks above + for _ in range(chunks_len - 2): + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 1}, {"Id": 2}], "nextRecordsUrl": next_page_url})) + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 3, "propertyB": "B"}, {"Id": 4, "propertyB": "B"}]})) + # 2 for 1 chunk above and 1 chunk had no next page + for _ in range(chunks_len - 2): + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 3}, {"Id": 4}]})) + + records = [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh)] assert records == [ {"Id": 1, "propertyA": "A", "propertyB": "B"}, {"Id": 2, "propertyA": "A", "propertyB": "B"}, @@ -801,7 +866,8 @@ def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties assert len(call.url) < Salesforce.REQUEST_SIZE_LIMITS -def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): +@pytest.mark.asyncio +async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties): stream = generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) chunks = list(stream.chunk_properties()) for chunk in chunks: @@ -809,86 +875,80 @@ def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_ assert stream.too_many_properties assert stream.primary_key assert type(stream) == RestSalesforceStream - url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" - requests_mock.get( - url, - [ - {"json": {"records": []}}, - ], - ) - records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) - assert records == [] + url = re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0/queryAll\??.*") + await stream.ensure_session() + + with aioresponses() as m: + m.get(url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"records": []})) + records = [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh)] + assert records == [] +@pytest.mark.asyncio @pytest.mark.parametrize( "status_code,response_json,log_message", [ ( - 400, - [{"errorCode": "INVALIDENTITY", "message": "Account is not supported by the Bulk API"}], - "Account is not supported by the Bulk API", + 400, + {"errorCode": "INVALIDENTITY", "message": "Account is not supported by the Bulk API"}, + "Account is not supported by the Bulk API", ), - (403, [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "API limit reached"}], "API limit reached"), - (400, [{"errorCode": "API_ERROR", "message": "API does not support query"}], "The stream 'Account' is not queryable,"), + (403, {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "API limit reached"}, "API limit reached"), + (400, {"errorCode": "API_ERROR", "message": "API does not support query"}, "The stream 'Account' is not queryable,"), ( - 400, - [{"errorCode": "LIMIT_EXCEEDED", "message": "Max bulk v2 query jobs (10000) per 24 hrs has been reached (10021)"}], - "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed.", + 400, + {"errorCode": "LIMIT_EXCEEDED", "message": "Max bulk v2 query jobs (10000) per 24 hrs has been reached (10021)"}, + "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed.", ), ], ) -def test_bulk_stream_error_in_logs_on_create_job(requests_mock, stream_config, stream_api, status_code, response_json, log_message, caplog): - """ """ +async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api, status_code, response_json, log_message, caplog): stream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query" - requests_mock.register_uri( - "POST", - url, - status_code=status_code, - json=response_json, - ) - query = "Select Id, Subject from Account" - with caplog.at_level(logging.ERROR): - assert stream.create_stream_job(query, url) is None, "this stream should be skipped" + + with aioresponses() as m: + m.post(url, status=status_code, callback=lambda *args, **kwargs: CallbackResult(status=status_code, payload=response_json, reason="")) + query = "Select Id, Subject from Account" + with caplog.at_level(logging.ERROR): + assert await stream.create_stream_job(query, url) is None, "this stream should be skipped" # check logs assert log_message in caplog.records[-1].message +@pytest.mark.asyncio @pytest.mark.parametrize( "status_code,response_json,error_message", [ ( 400, - [ - { - "errorCode": "TXN_SECURITY_METERING_ERROR", - "message": "We can't complete the action because enabled transaction security policies took too long to complete.", - } - ], + { + "errorCode": "TXN_SECURITY_METERING_ERROR", + "message": "We can't complete the action because enabled transaction security policies took too long to complete.", + }, 'A transient authentication error occurred. To prevent future syncs from failing, assign the "Exempt from Transaction Security" user permission to the authenticated user.', ), ], ) -def test_bulk_stream_error_on_wait_for_job(requests_mock, stream_config, stream_api, status_code, response_json, error_message): - +async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, status_code, response_json, error_message): stream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query/queryJobId" - requests_mock.register_uri( - "GET", - url, - status_code=status_code, - json=response_json, - ) - with pytest.raises(AirbyteTracedException) as e: - stream.wait_for_job(url=url) - assert e.value.message == error_message + + with aioresponses() as m: + m.get(url, status=status_code, callback=lambda *args, **kwargs: CallbackResult(status=status_code, payload=response_json, reason="")) + with pytest.raises(AirbyteTracedException) as e: + await stream.wait_for_job(url=url) + assert e.value.message == error_message + +@pytest.mark.asyncio() @freezegun.freeze_time("2023-01-01") -def test_bulk_stream_slices(stream_config_date_format, stream_api): +async def test_bulk_stream_slices(stream_config_date_format, stream_api): stream: BulkIncrementalSalesforceStream = generate_stream("FakeBulkStream", stream_config_date_format, stream_api) - stream_slices = list(stream.stream_slices(sync_mode=SyncMode.full_refresh)) + stream_slices = [s async for s in stream.stream_slices(sync_mode=SyncMode.full_refresh)] expected_slices = [] today = pendulum.today(tz="UTC") start_date = pendulum.parse(stream.start_date, tz="UTC") @@ -902,61 +962,68 @@ def test_bulk_stream_slices(stream_config_date_format, stream_api): start_date = start_date.add(days=stream.STREAM_SLICE_STEP) assert expected_slices == stream_slices + +@pytest.mark.asyncio @freezegun.freeze_time("2023-04-01") -def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog, requests_mock): - """Check that request params ignore records cursor and use start date from slice ONLY""" +async def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog): stream_config_date_format.update({"start_date": "2023-01-01"}) stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config_date_format, stream_api) + await stream.ensure_session() source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.streams = Mock() source.streams.return_value = [stream] + base_url = f"{stream.sf_api.instance_url}{stream.path()}" job_id_1 = "fake_job_1" - requests_mock.register_uri("GET", stream.path() + f"/{job_id_1}", [{"json": {"state": "JobComplete"}}]) - requests_mock.register_uri("DELETE", stream.path() + f"/{job_id_1}") - requests_mock.register_uri("GET", stream.path() + f"/{job_id_1}/results", text="Field1,LastModifiedDate,ID\ntest,2023-01-15,1") - requests_mock.register_uri("PATCH", stream.path() + f"/{job_id_1}") - job_id_2 = "fake_job_2" - requests_mock.register_uri("GET", stream.path() + f"/{job_id_2}", [{"json": {"state": "JobComplete"}}]) - requests_mock.register_uri("DELETE", stream.path() + f"/{job_id_2}") - requests_mock.register_uri( - "GET", stream.path() + f"/{job_id_2}/results", text="Field1,LastModifiedDate,ID\ntest,2023-04-01,2\ntest,2023-02-20,22" - ) - requests_mock.register_uri("PATCH", stream.path() + f"/{job_id_2}") - job_id_3 = "fake_job_3" - queries_history = requests_mock.register_uri( - "POST", stream.path(), [{"json": {"id": job_id_1}}, {"json": {"id": job_id_2}}, {"json": {"id": job_id_3}}] - ) - requests_mock.register_uri("GET", stream.path() + f"/{job_id_3}", [{"json": {"state": "JobComplete"}}]) - requests_mock.register_uri("DELETE", stream.path() + f"/{job_id_3}") - requests_mock.register_uri("GET", stream.path() + f"/{job_id_3}/results", text="Field1,LastModifiedDate,ID\ntest,2023-04-01,3") - requests_mock.register_uri("PATCH", stream.path() + f"/{job_id_3}") - logger = logging.getLogger("airbyte") - state = {"Account": {"LastModifiedDate": "2023-01-01T10:10:10.000Z"}} - bulk_catalog.streams.pop(1) - result = [i for i in source.read(logger=logger, config=stream_config_date_format, catalog=bulk_catalog, state=state)] + with aioresponses() as m: + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_1})) + m.get(base_url + f"/{job_id_1}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.delete(base_url + f"/{job_id_1}") + m.get(base_url + f"/{job_id_1}/results", + callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-01-15,1")) + m.patch(base_url + f"/{job_id_1}") + + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_2})) + m.get(base_url + f"/{job_id_2}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.delete(base_url + f"/{job_id_2}") + m.get(base_url + f"/{job_id_2}/results", + callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-04-01,2\ntest,2023-02-20,22")) + m.patch(base_url + f"/{job_id_2}") + + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_3})) + m.get(base_url + f"/{job_id_3}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.delete(base_url + f"/{job_id_3}") + m.get(base_url + f"/{job_id_3}/results", + callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-04-01,3")) + m.patch(base_url + f"/{job_id_3}") + + logger = logging.getLogger("airbyte") + state = {"Account": {"LastModifiedDate": "2023-01-01T10:10:10.000Z"}} + bulk_catalog.streams.pop(1) + result = [i for i in source.read(logger=logger, config=stream_config_date_format, catalog=bulk_catalog, state=state)] actual_state_values = [item.state.data.get("Account").get(stream.cursor_field) for item in result if item.type == Type.STATE] + queries_history = m.requests + # assert request params assert ( - "LastModifiedDate >= 2023-01-01T10:10:10.000+00:00 AND LastModifiedDate < 2023-01-31T10:10:10.000+00:00" - in queries_history.request_history[0].text + "LastModifiedDate >= 2023-01-01T10:10:10.000+00:00 AND LastModifiedDate < 2023-01-31T10:10:10.000+00:00" + in queries_history[("POST", URL(base_url))][0].kwargs["json"]["query"] ) assert ( - "LastModifiedDate >= 2023-01-31T10:10:10.000+00:00 AND LastModifiedDate < 2023-03-02T10:10:10.000+00:00" - in queries_history.request_history[1].text + "LastModifiedDate >= 2023-01-31T10:10:10.000+00:00 AND LastModifiedDate < 2023-03-02T10:10:10.000+00:00" + in queries_history[("POST", URL(base_url))][1].kwargs["json"]["query"] ) assert ( - "LastModifiedDate >= 2023-03-02T10:10:10.000+00:00 AND LastModifiedDate < 2023-04-01T00:00:00.000+00:00" - in queries_history.request_history[2].text + "LastModifiedDate >= 2023-03-02T10:10:10.000+00:00 AND LastModifiedDate < 2023-04-01T00:00:00.000+00:00" + in queries_history[("POST", URL(base_url))][2].kwargs["json"]["query"] ) # assert states - # if connector meets record with cursor `2023-04-01` out of current slice range 2023-01-31 <> 2023-03-02, we ignore all other values and set state to slice end_date expected_state_values = ["2023-01-15T00:00:00+00:00", "2023-03-02T10:10:10+00:00", "2023-04-01T00:00:00+00:00"] assert actual_state_values == expected_state_values @@ -975,30 +1042,26 @@ def test_request_params_substream(stream_config_date_format, stream_api): assert params == {"q": "SELECT LastModifiedDate, Id FROM ContentDocumentLink WHERE ContentDocumentId IN ('1','2')"} +@pytest.mark.asyncio @freezegun.freeze_time("2023-03-20") -def test_stream_slices_for_substream(stream_config, stream_api, requests_mock): - """Test BulkSalesforceSubStream for ContentDocumentLink (+ parent ContentDocument) - - ContentDocument return 1 record for each slice request. - Given start/end date leads to 3 date slice for ContentDocument, thus 3 total records - - ContentDocumentLink - It means that ContentDocumentLink should have 2 slices, with 2 and 1 records in each - """ +async def test_stream_slices_for_substream(stream_config, stream_api): stream_config['start_date'] = '2023-01-01' stream: BulkSalesforceSubStream = generate_stream("ContentDocumentLink", stream_config, stream_api) stream.SLICE_BATCH_SIZE = 2 # each ContentDocumentLink should contain 2 records from parent ContentDocument stream + await stream.ensure_session() job_id = "fake_job" - requests_mock.register_uri("POST", stream.path(), json={"id": job_id}) - requests_mock.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "JobComplete"}) - requests_mock.register_uri("GET", stream.path() + f"/{job_id}/results", [{"text": "Field1,LastModifiedDate,ID\ntest,2021-11-16,123", "headers": {"Sforce-Locator": "null"}}]) - requests_mock.register_uri("DELETE", stream.path() + f"/{job_id}") - - stream_slices = list(stream.stream_slices(sync_mode=SyncMode.full_refresh)) - assert stream_slices == [ - {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}, - {'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]}, - {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]} - ] - + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + m.post(base_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.get(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.get(base_url + f"/{job_id}/results", repeat=True, callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2021-11-16,123", headers={"Sforce-Locator": "null"})) + m.delete(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult()) + + stream_slices = [slice async for slice in stream.stream_slices(sync_mode=SyncMode.full_refresh)] + assert stream_slices == [ + {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}, + {'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]}, + {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]} + ] diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py index 75780d693822..6e38811903bf 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py @@ -6,7 +6,7 @@ import tracemalloc import pytest -import requests_mock +from aioresponses import CallbackResult, aioresponses from conftest import generate_stream from source_salesforce.streams import BulkIncrementalSalesforceStream @@ -26,17 +26,22 @@ "200k records", ], ) -def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): +@pytest.mark.asyncio +async def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() content = b'"Id","IsDeleted"' for _ in range(n_records): content += b'"0014W000027f6UwQAI","false"\n' - with requests_mock.Mocker() as m: - m.register_uri("GET", job_full_url_results, content=content) + def callback(url, **kwargs): + return CallbackResult(body=content) + + with aioresponses() as m: + m.get(job_full_url_results, status=200, callback=callback) tracemalloc.start() - tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) for x in stream.read_with_chunks(tmp_file, response_encoding): pass fs, fp = tracemalloc.get_traced_memory() @@ -44,3 +49,5 @@ def test_memory_download_data(stream_config, stream_api, n_records, first_size, assert first_size_in_mb < first_size assert first_peak_in_mb < first_peak + + await stream._session.close() From 475f33f116de586cb3f33b391afce706b805e10b Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 5 Jan 2024 23:41:06 -0500 Subject: [PATCH 60/74] Handle check_availability StopAsyncIteration --- .../airbyte_cdk/sources/abstract_source_async.py | 2 +- .../streams/http/availability_strategy_async.py | 6 ++++-- .../sources/streams/utils/stream_helper_async.py | 10 ++++------ 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 6b0169ad8c6e..bda6aa38a95d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -83,7 +83,7 @@ def __next__(self): raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") if not self.sentinels: self.thread.join() - raise StopIteration + raise StopAsyncIteration else: return self.__next__() else: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py index e647c63cf9b2..54136a50154b 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py @@ -34,7 +34,7 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, # Some streams need a stream slice to read records (e.g. if they have a SubstreamPartitionRouter) # Streams that don't need a stream slice will return `None` as their first stream slice. stream_slice = await get_first_stream_slice(stream) - except StopIteration: + except StopAsyncIteration: # If stream_slices has no `next()` item (Note - this is different from stream_slices returning [None]!) # This can happen when a substream's `stream_slices` method does a `for record in parent_records: yield ` # without accounting for the case in which the parent stream is empty. @@ -49,7 +49,7 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, try: async for _ in get_first_record_for_slice(stream, stream_slice): return True, None - except StopIteration: + except StopAsyncIteration: logger.info(f"Successfully connected to stream {stream.name}, but got 0 records.") return True, None except ClientResponseError as error: @@ -58,6 +58,8 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, reason = f"Unable to read {stream.name} stream. {reason}" return is_available, reason + return True, None + async def _handle_http_error( self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: ClientResponseError ) -> Tuple[bool, Optional[str]]: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py index bb16e3418f7b..47e2a6ecb7a2 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py @@ -9,16 +9,14 @@ from airbyte_cdk.sources.streams.core import StreamData -async def get_first_stream_slice(stream: AsyncStream) -> Optional[Mapping[str, Any]]: +async def get_first_stream_slice(stream: AsyncStream) -> Mapping[str, Any]: """ Gets the first stream_slice from a given stream's stream_slices. :param stream: stream - :raises StopIteration: if there is no first slice to return (the stream_slices generator is empty) + :raises StopAsyncIteration: if there is no first slice to return (the stream_slices generator is empty) :return: first stream slice from 'stream_slices' generator (`None` is a valid stream slice) """ - async for stream_slice in stream.stream_slices(cursor_field=stream.cursor_field, sync_mode=SyncMode.full_refresh): - return stream_slice - raise StopIteration(f"No slices in stream {stream.name}") + return await anext(stream.stream_slices(cursor_field=stream.cursor_field, sync_mode=SyncMode.full_refresh)) async def get_first_record_for_slice(stream: AsyncStream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: @@ -26,7 +24,7 @@ async def get_first_record_for_slice(stream: AsyncStream, stream_slice: Optional Gets the first record for a stream_slice of a stream. :param stream: stream :param stream_slice: stream_slice - :raises StopIteration: if there is no first record to return (the read_records generator is empty) + :raises StopAsyncIteration: if there is no first record to return (the read_records generator is empty) :return: StreamData containing the first record in the slice """ # We wrap the return output of read_records() because some implementations return types that are iterable, From 67fe34ea9983bdf4611f06b9c8b605500bb5cdf2 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Fri, 5 Jan 2024 23:55:20 -0500 Subject: [PATCH 61/74] cleanup --- .../airbyte_cdk/sources/streams/async_call_rate.py | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py index 222367ade5dc..d690c153c152 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py @@ -2,21 +2,15 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # -import abc -import datetime import logging -from typing import TYPE_CHECKING, Any +from typing import Any import aiohttp import aiohttp_client_cache from airbyte_cdk.sources.streams.call_rate import AbstractAPIBudget -# prevents mypy from complaining about missing session attributes in LimiterMixin -if TYPE_CHECKING: - MIXIN_BASE = aiohttp.ClientSession -else: - MIXIN_BASE = object +MIXIN_BASE = aiohttp.ClientSession logger = logging.getLogger("airbyte") From 96d1a78715b6b57aa58df9be21c2212ac7cc771c Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 6 Jan 2024 12:53:48 -0500 Subject: [PATCH 62/74] StopIteration in SourceReader.__next__ should not be StopAsyncIteration --- .../python/airbyte_cdk/sources/abstract_source_async.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index bda6aa38a95d..d84b9757c3e9 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -83,7 +83,7 @@ def __next__(self): raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") if not self.sentinels: self.thread.join() - raise StopAsyncIteration + raise StopIteration else: return self.__next__() else: @@ -92,8 +92,9 @@ def __next__(self): loop.create_task(self.cleanup()) async def cleanup(self): - for session in self.sessions.values(): - await session.close() + pass + # for session in self.sessions.values(): + # await session.close() class AsyncAbstractSource(AbstractSource, ABC): From 537010d4771d661749c77ae923f0c02b44fc66d4 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 6 Jan 2024 17:02:43 -0500 Subject: [PATCH 63/74] Clean up error handling; CDK passing, Salesforce 2 failing due to exception type not being caught --- .../sources/abstract_source_async.py | 55 +++++++++---------- .../sources/streams/http/http_async.py | 21 +++---- .../sources/streams/http/test_http_async.py | 35 +++++------- .../sources/test_abstract_source_async.py | 11 +--- .../source_salesforce/streams.py | 6 +- 5 files changed, 56 insertions(+), 72 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index d84b9757c3e9..1eec5ae29ed7 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -51,28 +51,21 @@ def __init__(self, logger: logging.Logger, queue: Queue, sentinels: Dict[str, Se self.reader_fn = reader_fn self.reader_args = args self.sessions: Dict[str, aiohttp.ClientSession] = {} - self.error = None - self.error_lock = Lock() self.thread = Thread(target=self._start_reader_thread) self.thread.start() def _start_reader_thread(self): - try: - asyncio.run(self.reader_fn(*self.reader_args)) - except Exception as e: # TODO: is this definitely needed? - with self.error_lock: - self.error = (e, traceback.format_exc()) + asyncio.run(self.reader_fn(*self.reader_args)) def __next__(self): loop = asyncio.get_event_loop() try: - with self.error_lock: - if self.error: - exception, traceback_str = self.error - self.logger.error(f"An error occurred in the async thread: {traceback_str}") - raise exception item = self.queue.get(timeout=DEFAULT_TIMEOUT) + if isinstance(item, Exception): + self.logger.error(f"An error occurred in the async thread: {item}") + self.thread.join() + raise item if isinstance(item, Sentinel): # Sessions can only be closed once items in the stream have all been dequeued if session := self.sessions.pop(item.name, None): @@ -91,10 +84,15 @@ def __next__(self): finally: loop.create_task(self.cleanup()) + def drain(self): + while not self.queue.empty(): + yield self.queue.get() + self.thread.join() + async def cleanup(self): - pass - # for session in self.sessions.values(): - # await session.close() + # pass + for session in self.sessions.values(): + await session.close() class AsyncAbstractSource(AbstractSource, ABC): @@ -169,6 +167,11 @@ def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str self.reader = SourceReader(logger, self.queue, streams_in_progress_sentinels, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) for record in self.reader: yield record + print("DRAINING __________________________") + for record in self.reader.drain(): + if isinstance(record, Exception): + raise record + yield record def _assert_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream]): for configured_stream in catalog.streams: @@ -185,7 +188,7 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance pending_tasks = set() n_started, n_streams = 0, len(catalog.streams) streams_iterator = iter(catalog.streams) - exceptions = [] + exceptions = False while (pending_tasks or n_started < n_streams) and not exceptions: while len(pending_tasks) < self.session_limit and (configured_stream := next(streams_iterator, None)): @@ -201,13 +204,11 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance done, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) for task in done: - if task.exception(): + if exc := task.exception(): for remaining_task in pending_tasks: await remaining_task.cancel() - exceptions.append(task.exception()) - - if exceptions: - pass + self.queue.put(exc) + exceptions = True async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): try: @@ -235,9 +236,7 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) except AirbyteTracedException as e: - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) - with self.reader.error_lock: - self.reader.error = (e, traceback.format_exc()) + raise stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE) except Exception as e: for message in self._emit_queued_messages(): self.queue.put(message) @@ -246,11 +245,9 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) display_message = await stream_instance.get_error_display_message(e) if display_message: - exc = AirbyteTracedException.from_exception(e, message=display_message) - with self.reader.error_lock: - self.reader.error = (exc, traceback.format_exc()) - with self.reader.error_lock: - self.reader.error = (e, traceback.format_exc()) + raise AirbyteTracedException.from_exception(e, message=display_message) + else: + raise e finally: timer.finish_event() logger.info(f"Finished syncing {configured_stream.stream.name}") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py index 3c0dfccefd20..87cb68974b2b 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py @@ -292,7 +292,7 @@ async def send(): return await send() @classmethod - async def parse_response_error_message(cls, response: aiohttp.ClientResponse) -> Optional[str]: + async def parse_response_error_message(cls, exception: aiohttp.ClientResponseError) -> Optional[str]: """ Parses the raw response object from a failed request into a user-friendly error message. By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. @@ -321,13 +321,10 @@ def _try_get_error(value: Optional[JsonType]) -> Optional[str]: return None try: - if hasattr(response, "_response_json"): - return response._response_json - try: - body = await response.json() - return _try_get_error(body) - except AttributeError: - pass + if hasattr(exception, "_response_error"): + return _try_get_error(exception._response_error) + else: + raise NotImplementedError("`_response_error` is expected but was not set on the response; `handle_response_with_error` should be used prior to processing the exception") except json.JSONDecodeError: return None @@ -416,7 +413,11 @@ async def handle_response_with_error(self, response: aiohttp.ClientResponse) -> try: error_json = await response.json() except (json.JSONDecodeError, aiohttp.ContentTypeError): - error_json = {} + error_json = None + except Exception as exc: + raise NotImplementedError(f"Unexpected!!!!!!!! {exc}") # TODO + self.logger.error(f"Unable to get error json from response: {exc}") + error_json = None exc = aiohttp.ClientResponseError( response.request_info, @@ -425,8 +426,8 @@ async def handle_response_with_error(self, response: aiohttp.ClientResponse) -> message=response.reason, headers=response.headers, ) - exc._error_json = error_json # https://github.com/aio-libs/aiohttp/issues/3248 text = await response.text() + exc._response_error = error_json or text # https://github.com/aio-libs/aiohttp/issues/3248 self.logger.error(text) raise exc diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py index 29345b94bccb..33616db16a1b 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py @@ -624,6 +624,7 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): loop.run_until_complete(stream._session.close()) +@pytest.mark.asyncio @pytest.mark.parametrize( "api_response, expected_message", [ @@ -645,36 +646,28 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): ({}, None), ], ) -def test_default_parse_response_error_message(api_response: dict, expected_message: Optional[str]): +async def test_default_parse_response_error_message(api_response: dict, expected_message: Optional[str]): stream = StubBasicReadHttpStream() - loop = asyncio.get_event_loop() response = MagicMock() - response.json.return_value = _get_response(api_response) + response._response_error = api_response - message = loop.run_until_complete(stream.parse_response_error_message(response)) + message = await stream.parse_response_error_message(response) assert message == expected_message -async def _get_response(response): - return response - - -def test_default_parse_response_error_message_not_json(): +@pytest.mark.asyncio +async def test_default_parse_response_error_message_not_json(): stream = StubBasicReadHttpStream() - loop = asyncio.get_event_loop() - loop.run_until_complete(stream.ensure_session()) - - req = aiohttp.ClientRequest("GET", URL("mock://test.com/not_json")) - - def callback(url, **kwargs): - return CallbackResult(body="this is not json") + await stream.ensure_session() + url = "mock://test.com/not_json" with aioresponses() as m: - m.get("mock://test.com/not_json", callback=callback) - response = loop.run_until_complete(stream._send_request(req, {})) - message = loop.run_until_complete(stream.parse_response_error_message(response)) - assert message is None - loop.run_until_complete(stream._session.close()) + m.get(url, callback=lambda *_, **__: CallbackResult(status=400, body="this is not json")) + with pytest.raises(aiohttp.ClientResponseError): + response = await stream._send_request(aiohttp.ClientRequest("GET", URL(url)), {}) + message = await stream.parse_response_error_message(response) + assert message is None + await stream._session.close() def test_default_get_error_display_message_handles_http_error(mocker): diff --git a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py b/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py index 5a03f7cd57eb..c82ff0c6a285 100644 --- a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py +++ b/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py @@ -312,7 +312,7 @@ async def read_records_with_error(*args, **kwargs): raise RuntimeError("oh no!") -def test_read_stream_with_error_no_display_message(mocker): +def test_read_stream_with_error_gets_display_message(mocker): stream = MockStream(name="my_stream") mocker.patch.object(MockStream, "get_json_schema", return_value={}) stream.read_records = read_records_with_error @@ -320,19 +320,12 @@ def test_read_stream_with_error_no_display_message(mocker): source = MockSource(streams=[stream]) catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) + # without get_error_display_message with pytest.raises(RuntimeError, match="oh no!"): list(source.read(logger, {}, catalog)) - -def test_read_stream_with_error_with_display_message(mocker): - stream = MockStream(name="my_stream") - mocker.patch.object(MockStream, "get_json_schema", return_value={}) - stream.read_records = read_records_with_error stream.get_error_display_message = AsyncMock(return_value="my message") - source = MockSource(streams=[stream]) - catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) - with pytest.raises(AirbyteTracedException, match="oh no!") as exc: list(source.read(logger, {}, catalog)) assert exc.value.message == "my message" diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 087a3fcd0965..2038a04cd0c7 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -383,9 +383,9 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: # updated query: "Select Name, (Select Subject,ActivityType from ActivityHistories) from Contact" # The second variant forces customisation for every case (ActivityHistory, ActivityHistories etc). # And the main problem is these subqueries doesn't support CSV response format. - if not hasattr(error, "_error_json"): + if not hasattr(error, "_response_error"): raise NotImplementedError("!!!!!!!!!!!!! this didn't use `handle_response_with_error`") - error_data = error._error_json or {} + error_data = error._response_error or {} error_code = error_data.get("errorCode") error_message = error_data.get("message", "") if error_message == "Selecting compound data not supported in Bulk Query" or ( @@ -439,7 +439,7 @@ async def wait_for_job(self, url: str) -> str: try: job_info = await (await self._send_http_request("GET", url=url)).json() except aiohttp.ClientResponseError as error: - error_data = error._error_json + error_data = error._response_error error_code = error_data.get("errorCode") error_message = error_data.get("message", "") if ( From b8ce8db9ca7463c705e219a166332ad5436fd9d6 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 6 Jan 2024 17:28:02 -0500 Subject: [PATCH 64/74] Fix error handling; all Salesforce & CDK tests & comparison with master pass --- .../python/airbyte_cdk/sources/abstract_source_async.py | 3 ++- .../connectors/source-salesforce/source_salesforce/source.py | 2 +- .../connectors/source-salesforce/unit_tests/api_test.py | 4 ---- 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py index 1eec5ae29ed7..956151fbbc23 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py @@ -236,7 +236,8 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) except AirbyteTracedException as e: - raise stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE) + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + raise e except Exception as e: for message in self._emit_queued_messages(): self.queue.put(message) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index 9b0d9b3e4926..9ed994f27b9b 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -226,7 +226,7 @@ async def _read_stream( async for record in super()._read_stream(logger, stream_instance, configured_stream, state_manager, internal_config): yield record except aiohttp.ClientResponseError as error: - error_data = error._error_json if hasattr(error, "_error_json") else {} + error_data = error._response_error if hasattr(error, "_response_error") else {} error_code = error_data.get("errorCode") url = error.request_info.url if error.status == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index cd24b089b100..1e394362de8d 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -495,8 +495,6 @@ def cb(response): state_record = [item for item in result if item.type == Type.STATE][0] assert state_record.state.data["Account"]["LastModifiedDate"] == "2021-10-05T00:00:00+00:00" # state checkpoint interval is 5. - await stream_1._session.close() - await stream_2._session.close() @pytest.mark.asyncio @@ -588,8 +586,6 @@ async def check_availability(*args, **kwargs): state_record = [item for item in result if item.type == Type.STATE][0] assert state_record.state.data["KnowledgeArticle"]["LastModifiedDate"] == "2021-11-17T00:00:00+00:00" - await stream_1._session.close() - await stream_2._session.close() @pytest.mark.asyncio From 145ee23921c1be79f555511040cf5eb5ad875b0d Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 6 Jan 2024 18:46:19 -0500 Subject: [PATCH 65/74] Reorg into separate async_cdk module --- .../python/airbyte_cdk/sources/__init__.py | 3 +- .../{ => async_cdk}/abstract_source_async.py | 2 +- .../sources/async_cdk/streams/__init__.py | 0 .../streams/async_call_rate.py | 0 .../streams/availability_strategy_async.py | 0 .../{ => async_cdk}/streams/core_async.py | 0 .../http/availability_strategy_async.py | 5 +- .../streams/http/exceptions_async.py | 0 .../streams/http/http_async.py | 10 +- .../streams/http/rate_limiting_async.py | 0 .../streams/utils/stream_helper_async.py | 2 +- .../airbyte_cdk/sources/streams/__init__.py | 3 +- .../concurrent/abstract_stream_async.py | 83 ---- .../streams/concurrent/adapters_async.py | 429 ------------------ .../concurrent/default_stream_async.py | 80 ---- .../concurrent/partitions/partition_async.py | 63 --- .../partitions/partition_generator_async.py | 18 - .../unit_tests/sources/async_cdk/__init__.py | 0 .../async_cdk/scenario_based/__init__.py | 0 .../async_concurrent_stream_scenarios.py | 0 .../async_concurrent_stream_source_builder.py | 0 .../test_async_concurrent_scenarios.py | 0 .../sources/async_cdk/streams/__init__.py | 0 .../async_cdk/streams/http/__init__.py | 0 .../http/test_availability_strategy_async.py | 6 +- .../streams/http/test_http_async.py | 4 +- .../test_availability_strategy_async.py | 2 +- .../streams/test_streams_core_async.py | 2 +- .../async_cdk/streams/utils/__init__.py | 0 .../utils/test_rate_limiting_async.py | 2 +- .../test_abstract_source_async.py | 5 +- .../availability_strategy.py | 2 +- .../source_salesforce/source.py | 2 +- .../source_salesforce/streams.py | 2 +- .../source-salesforce/unit_tests/api_test.py | 2 +- 35 files changed, 27 insertions(+), 700 deletions(-) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/abstract_source_async.py (99%) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/__init__.py rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/async_call_rate.py (100%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/availability_strategy_async.py (100%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/core_async.py (100%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/http/availability_strategy_async.py (96%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/http/exceptions_async.py (100%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/http/http_async.py (97%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/http/rate_limiting_async.py (100%) rename airbyte-cdk/python/airbyte_cdk/sources/{ => async_cdk}/streams/utils/stream_helper_async.py (95%) delete mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py delete mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py delete mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py delete mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py delete mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py create mode 100644 airbyte-cdk/python/unit_tests/sources/async_cdk/__init__.py create mode 100644 airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/__init__.py rename airbyte-cdk/python/unit_tests/sources/{concurrent_source => async_cdk/scenario_based}/async_concurrent_stream_scenarios.py (100%) rename airbyte-cdk/python/unit_tests/sources/{streams/concurrent/scenarios => async_cdk/scenario_based}/async_concurrent_stream_source_builder.py (100%) rename airbyte-cdk/python/unit_tests/sources/{concurrent_source => async_cdk/scenario_based}/test_async_concurrent_scenarios.py (100%) create mode 100644 airbyte-cdk/python/unit_tests/sources/async_cdk/streams/__init__.py create mode 100644 airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/__init__.py rename airbyte-cdk/python/unit_tests/sources/{ => async_cdk}/streams/http/test_availability_strategy_async.py (95%) rename airbyte-cdk/python/unit_tests/sources/{ => async_cdk}/streams/http/test_http_async.py (99%) rename airbyte-cdk/python/unit_tests/sources/{ => async_cdk}/streams/test_availability_strategy_async.py (95%) rename airbyte-cdk/python/unit_tests/sources/{ => async_cdk}/streams/test_streams_core_async.py (98%) create mode 100644 airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/__init__.py rename airbyte-cdk/python/unit_tests/{ => sources/async_cdk/streams}/utils/test_rate_limiting_async.py (88%) rename airbyte-cdk/python/unit_tests/sources/{ => async_cdk}/test_abstract_source_async.py (99%) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/__init__.py index 631c2b0762fa..396513539fcd 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/__init__.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/__init__.py @@ -5,7 +5,6 @@ import dpath.options from .abstract_source import AbstractSource -from .abstract_source_async import AsyncAbstractSource from .config import BaseConfig from .source import Source @@ -20,4 +19,4 @@ # this will not be thread-safe. dpath.options.ALLOW_EMPTY_STRING_KEYS = True -__all__ = ["AbstractSource", "AsyncAbstractSource", "BaseConfig", "Source"] +__all__ = ["AbstractSource", "BaseConfig", "Source"] diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py similarity index 99% rename from airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py index 956151fbbc23..2c60a36640b9 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py @@ -24,7 +24,7 @@ from airbyte_cdk.models import Type as MessageType from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.abstract_source import AbstractSource -from airbyte_cdk.sources.streams import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.http import HttpStream from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/async_call_rate.py similarity index 100% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/async_call_rate.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/async_call_rate.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/availability_strategy_async.py similarity index 100% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/availability_strategy_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/availability_strategy_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py similarity index 100% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/core_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py similarity index 96% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py index 54136a50154b..722f4bedaead 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py @@ -8,9 +8,10 @@ import requests from aiohttp import ClientResponseError -from airbyte_cdk.sources.streams import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy -from airbyte_cdk.sources.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice + if typing.TYPE_CHECKING: from airbyte_cdk.sources import Source diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py similarity index 100% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py similarity index 97% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py index 87cb68974b2b..1204ce4cc7c6 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py @@ -13,18 +13,18 @@ import aiohttp import aiohttp_client_cache from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.async_cdk.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy +from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE -from airbyte_cdk.sources.streams import AsyncStream -from airbyte_cdk.sources.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.auth import NoAuth -from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy +from airbyte_cdk.sources.streams.http.auth.core import HttpAuthenticator from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream -from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH -from .auth.core import HttpAuthenticator from .rate_limiting_async import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py similarity index 100% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py similarity index 95% rename from airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py rename to airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py index 47e2a6ecb7a2..293c12ab620a 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py @@ -5,7 +5,7 @@ from typing import Any, Mapping, Optional from airbyte_cdk.models import SyncMode -from airbyte_cdk.sources.streams import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.streams.core import StreamData diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py index f50ec9119bd9..9326fd1bdca7 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py @@ -4,6 +4,5 @@ # Initialize Streams Package from .core import IncrementalMixin, Stream -from .core_async import AsyncStream -__all__ = ["IncrementalMixin", "Stream", "AsyncStream"] +__all__ = ["IncrementalMixin", "Stream"] diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py deleted file mode 100644 index 4f1281d37cdd..000000000000 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/abstract_stream_async.py +++ /dev/null @@ -1,83 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -from abc import ABC, abstractmethod -from typing import Any, Iterable, Mapping, Optional - -from airbyte_cdk.models import AirbyteStream -from airbyte_cdk.sources.streams.concurrent.availability_strategy import StreamAvailability -from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition -from deprecated.classic import deprecated - - -@deprecated("This class is experimental. Use at your own risk.") -class AsyncAbstractStream(ABC): - """ - AbstractStream is an experimental interface for streams developed as part of the Concurrent CDK. - This interface is not yet stable and may change in the future. Use at your own risk. - - Why create a new interface instead of adding concurrency capabilities the existing Stream? - We learnt a lot since the initial design of the Stream interface, and we wanted to take the opportunity to improve. - - High level, the changes we are targeting are: - - Removing superfluous or leaky parameters from the methods' interfaces - - Using composition instead of inheritance to add new capabilities - - To allow us to iterate fast while ensuring backwards compatibility, we are creating a new interface with a facade object that will bridge the old and the new interfaces. - Source connectors that wish to leverage concurrency need to implement this new interface. An example will be available shortly - - Current restrictions on sources that implement this interface. Not all of these restrictions will be lifted in the future, but most will as we iterate on the design. - - Only full refresh is supported. This will be addressed in the future. - - The read method does not accept a cursor_field. Streams must be internally aware of the cursor field to use. User-defined cursor fields can be implemented by modifying the connector's main method to instantiate the streams with the configured cursor field. - - Streams cannot return user-friendly messages by overriding Stream.get_error_display_message. This will be addressed in the future. - - The Stream's behavior cannot depend on a namespace - - TypeTransformer is not supported. This will be addressed in the future. - - Nested cursor and primary keys are not supported - """ - - @abstractmethod - async def generate_partitions(self) -> Iterable[Partition]: - """ - Generates the partitions that will be read by this stream. - :return: An iterable of partitions. - """ - - @property - @abstractmethod - def name(self) -> str: - """ - :return: The stream name - """ - - @property - @abstractmethod - def cursor_field(self) -> Optional[str]: - """ - Override to return the default cursor field used by this stream e.g: an API entity might always use created_at as the cursor field. - :return: The name of the field used as a cursor. Nested cursor fields are not supported. - """ - - @abstractmethod - def check_availability(self) -> StreamAvailability: - """ - :return: The stream's availability - """ - - @abstractmethod - def get_json_schema(self) -> Mapping[str, Any]: - """ - :return: A dict of the JSON schema representing this stream. - """ - - @abstractmethod - def as_airbyte_stream(self) -> AirbyteStream: - """ - :return: A dict of the JSON schema representing this stream. - """ - - @abstractmethod - def log_stream_sync_configuration(self) -> None: - """ - Logs the stream's configuration for debugging purposes. - """ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py deleted file mode 100644 index 8993b50e90dc..000000000000 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/adapters_async.py +++ /dev/null @@ -1,429 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -import copy -import json -import logging -from functools import lru_cache -from typing import Any, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Union - -from airbyte_cdk.models import AirbyteStream, SyncMode -from airbyte_cdk.sources import AbstractSource, Source -from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager -from airbyte_cdk.sources.message import MessageRepository -from airbyte_cdk.sources.streams import AsyncStream -from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy -from airbyte_cdk.sources.streams.concurrent.abstract_stream import AbstractStream -from airbyte_cdk.sources.streams.concurrent.availability_strategy import ( - AbstractAvailabilityStrategy, - StreamAvailability, - StreamAvailable, - StreamUnavailable, -) -from airbyte_cdk.sources.streams.concurrent.cursor import Cursor, NoopCursor -from airbyte_cdk.sources.streams.concurrent.default_stream_async import AsyncDefaultStream -from airbyte_cdk.sources.streams.concurrent.exceptions import ExceptionWithDisplayMessage -from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition -from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator -from airbyte_cdk.sources.streams.concurrent.partitions.record import Record -from airbyte_cdk.sources.streams.core import StreamData -from airbyte_cdk.sources.utils.schema_helpers import InternalConfig -from airbyte_cdk.sources.utils.slice_logger import SliceLogger -from deprecated.classic import deprecated - -""" -This module contains adapters to help enabling concurrency on Stream objects without needing to migrate to AbstractStream -""" - - -@deprecated("This class is experimental. Use at your own risk.") -class AsyncStreamFacade(AsyncStream): - """ - The StreamFacade is a Stream that wraps an AbstractStream and exposes it as a Stream. - - All methods either delegate to the wrapped AbstractStream or provide a default implementation. - The default implementations define restrictions imposed on Streams migrated to the new interface. For instance, only source-defined cursors are supported. - """ - - @classmethod - def create_from_stream( - cls, - stream: AsyncStream, - source: AbstractSource, - logger: logging.Logger, - state: Optional[MutableMapping[str, Any]], - cursor: Cursor, - ) -> AsyncStream: - """ - Create a ConcurrentStream from a Stream object. - :param source: The source - :param stream: The stream - :param max_workers: The maximum number of worker thread to use - :return: - """ - pk = cls._get_primary_key_from_stream(stream.primary_key) - cursor_field = cls._get_cursor_field_from_stream(stream) - - if not source.message_repository: - raise ValueError( - "A message repository is required to emit non-record messages. Please set the message repository on the source." - ) - - message_repository = source.message_repository - return AsyncStreamFacade( - AsyncDefaultStream( - partition_generator=AsyncStreamPartitionGenerator( - stream, - message_repository, - SyncMode.full_refresh if isinstance(cursor, NoopCursor) else SyncMode.incremental, - [cursor_field] if cursor_field is not None else None, - state, - cursor, - ), - name=stream.name, - namespace=stream.namespace, - json_schema=stream.get_json_schema(), - availability_strategy=AsyncStreamAvailabilityStrategy(stream, source), - primary_key=pk, - cursor_field=cursor_field, - logger=logger, - ), - stream, - cursor, - slice_logger=source._slice_logger, - logger=logger, - ) - - @property - def state(self) -> MutableMapping[str, Any]: - raise NotImplementedError("This should not be called as part of the Concurrent CDK code. Please report the problem to Airbyte") - - @state.setter - def state(self, value: Mapping[str, Any]) -> None: - if "state" in dir(self._legacy_stream): - self._legacy_stream.state = value # type: ignore # validating `state` is attribute of stream using `if` above - - @classmethod - def _get_primary_key_from_stream(cls, stream_primary_key: Optional[Union[str, List[str], List[List[str]]]]) -> List[str]: - if stream_primary_key is None: - return [] - elif isinstance(stream_primary_key, str): - return [stream_primary_key] - elif isinstance(stream_primary_key, list): - if len(stream_primary_key) > 0 and all(isinstance(k, str) for k in stream_primary_key): - return stream_primary_key # type: ignore # We verified all items in the list are strings - else: - raise ValueError(f"Nested primary keys are not supported. Found {stream_primary_key}") - else: - raise ValueError(f"Invalid type for primary key: {stream_primary_key}") - - @classmethod - def _get_cursor_field_from_stream(cls, stream: AsyncStream) -> Optional[str]: - if isinstance(stream.cursor_field, list): - if len(stream.cursor_field) > 1: - raise ValueError(f"Nested cursor fields are not supported. Got {stream.cursor_field} for {stream.name}") - elif len(stream.cursor_field) == 0: - return None - else: - return stream.cursor_field[0] - else: - return stream.cursor_field - - def __init__(self, stream: AbstractStream, legacy_stream: AsyncStream, cursor: Cursor, slice_logger: SliceLogger, logger: logging.Logger): - """ - :param stream: The underlying AbstractStream - """ - self._abstract_stream = stream - self._legacy_stream = legacy_stream - self._cursor = cursor - self._slice_logger = slice_logger - self._logger = logger - - async def read_full_refresh( - self, - cursor_field: Optional[List[str]], - logger: logging.Logger, - slice_logger: SliceLogger, - ) -> Iterable[StreamData]: - """ - Read full refresh. Delegate to the underlying AbstractStream, ignoring all the parameters - :param cursor_field: (ignored) - :param logger: (ignored) - :param slice_logger: (ignored) - :return: Iterable of StreamData - """ - async for record in self._read_records(): - yield record - - async def read_incremental( - self, - cursor_field: Optional[List[str]], - logger: logging.Logger, - slice_logger: SliceLogger, - stream_state: MutableMapping[str, Any], - state_manager: ConnectorStateManager, - per_stream_state_enabled: bool, - internal_config: InternalConfig, - ) -> Iterable[StreamData]: - async for record in self._read_records(): - yield record - - async def read_records( - self, - sync_mode: SyncMode, - cursor_field: Optional[List[str]] = None, - stream_slice: Optional[Mapping[str, Any]] = None, - stream_state: Optional[Mapping[str, Any]] = None, - ) -> Iterable[StreamData]: - async for record in self._read_records(): - yield record - - async def _read_records(self) -> Iterable[StreamData]: - async for partition in self._abstract_stream.generate_partitions(): - if self._slice_logger.should_log_slice_message(self._logger): - yield self._slice_logger.create_slice_log_message(partition.to_slice()) - async for record in partition.read(): - yield record.data - - @property - def name(self) -> str: - return self._abstract_stream.name - - @property - def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: - # This method is not expected to be called directly. It is only implemented for backward compatibility with the old interface - return self.as_airbyte_stream().source_defined_primary_key # type: ignore # source_defined_primary_key is known to be an Optional[List[List[str]]] - - @property - def cursor_field(self) -> Union[str, List[str]]: - if self._abstract_stream.cursor_field is None: - return [] - else: - return self._abstract_stream.cursor_field - - @property - def source_defined_cursor(self) -> bool: - # Streams must be aware of their cursor at instantiation time - return True - - @lru_cache(maxsize=None) - def get_json_schema(self) -> Mapping[str, Any]: - return self._abstract_stream.get_json_schema() - - @property - def supports_incremental(self) -> bool: - return self._legacy_stream.supports_incremental - - async def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: - """ - Verifies the stream is available. Delegates to the underlying AbstractStream and ignores the parameters - :param logger: (ignored) - :param source: (ignored) - :return: - """ - availability = await self._abstract_stream.check_availability() - return availability.is_available(), availability.message() - - def get_error_display_message(self, exception: BaseException) -> Optional[str]: - """ - Retrieves the user-friendly display message that corresponds to an exception. - This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. - - A display message will be returned if the exception is an instance of ExceptionWithDisplayMessage. - - :param exception: The exception that was raised - :return: A user-friendly message that indicates the cause of the error - """ - if isinstance(exception, ExceptionWithDisplayMessage): - return exception.display_message - else: - return None - - def as_airbyte_stream(self) -> AirbyteStream: - return self._abstract_stream.as_airbyte_stream() - - def log_stream_sync_configuration(self) -> None: - self._abstract_stream.log_stream_sync_configuration() - - -class AsyncStreamPartition(Partition): - """ - This class acts as an adapter between the new Partition interface and the Stream's stream_slice interface - - StreamPartitions are instantiated from a Stream and a stream_slice. - - This class can be used to help enable concurrency on existing connectors without having to rewrite everything as AbstractStream. - In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. - """ - - def __init__( - self, - stream: AsyncStream, - _slice: Optional[Mapping[str, Any]], - message_repository: MessageRepository, - sync_mode: SyncMode, - cursor_field: Optional[List[str]], - state: Optional[MutableMapping[str, Any]], - cursor: Cursor, - ): - """ - :param stream: The stream to delegate to - :param _slice: The partition's stream_slice - :param message_repository: The message repository to use to emit non-record messages - """ - self._stream = stream - self._slice = _slice - self._message_repository = message_repository - self._sync_mode = sync_mode - self._cursor_field = cursor_field - self._state = state - self._cursor = cursor - self._is_closed = False - - async def read(self) -> Iterable[Record]: - """ - Read messages from the stream. - If the StreamData is a Mapping, it will be converted to a Record. - Otherwise, the message will be emitted on the message repository. - """ - try: - # using `stream_state=self._state` have a very different behavior than the current one as today the state is updated slice - # by slice incrementally. We don't have this guarantee with Concurrent CDK. For HttpStream, `stream_state` is passed to: - # * fetch_next_page - # * parse_response - # Both are not used for Stripe so we should be good for the first iteration of Concurrent CDK. However, Stripe still do - # `if not stream_state` to know if it calls the Event stream or not - async for record_data in self._stream.read_records( - cursor_field=self._cursor_field, - sync_mode=SyncMode.full_refresh, - stream_slice=copy.deepcopy(self._slice), - stream_state=self._state, - ): - if isinstance(record_data, Mapping): - data_to_return = dict(record_data) - self._stream.transformer.transform(data_to_return, self._stream.get_json_schema()) - record = Record(data_to_return, self._stream.name) - self._cursor.observe(record) - yield Record(data_to_return, self._stream.name) - else: - self._message_repository.emit_message(record_data) - except Exception as e: - display_message = await self._stream.get_error_display_message(e) - if display_message: - raise ExceptionWithDisplayMessage(display_message) from e - else: - raise e - - def to_slice(self) -> Optional[Mapping[str, Any]]: - return self._slice - - def __hash__(self) -> int: - if self._slice: - # Convert the slice to a string so that it can be hashed - s = json.dumps(self._slice, sort_keys=True) - return hash((self._stream.name, s)) - else: - return hash(self._stream.name) - - def stream_name(self) -> str: - return self._stream.name - - def close(self) -> None: - self._cursor.close_partition(self) - self._is_closed = True - - def is_closed(self) -> bool: - return self._is_closed - - def __repr__(self) -> str: - return f"StreamPartition({self._stream.name}, {self._slice})" - - -class AsyncStreamPartitionGenerator(PartitionGenerator): - """ - This class acts as an adapter between the new PartitionGenerator and Stream.stream_slices - - This class can be used to help enable concurrency on existing connectors without having to rewrite everything as AbstractStream. - In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. - """ - - def __init__( - self, - stream: AsyncStream, - message_repository: MessageRepository, - sync_mode: SyncMode, - cursor_field: Optional[List[str]], - state: Optional[MutableMapping[str, Any]], - cursor: Cursor, - ): - """ - :param stream: The stream to delegate to - :param message_repository: The message repository to use to emit non-record messages - """ - self.message_repository = message_repository - self._stream = stream - self._sync_mode = sync_mode - self._cursor_field = cursor_field - self._state = state - self._cursor = cursor - - async def generate(self) -> Iterable[Partition]: - async for s in self._stream.stream_slices(sync_mode=self._sync_mode, cursor_field=self._cursor_field, stream_state=self._state): - yield AsyncStreamPartition( - self._stream, copy.deepcopy(s), self.message_repository, self._sync_mode, self._cursor_field, self._state, self._cursor - ) - - -@deprecated("This class is experimental. Use at your own risk.") -class AsyncAvailabilityStrategyFacade(AvailabilityStrategy): - def __init__(self, abstract_availability_strategy: AbstractAvailabilityStrategy): - self._abstract_availability_strategy = abstract_availability_strategy - - async def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional[Source]) -> Tuple[bool, Optional[str]]: - """ - Checks stream availability. - - Important to note that the stream and source parameters are not used by the underlying AbstractAvailabilityStrategy. - - :param stream: (unused) - :param logger: logger object to use - :param source: (unused) - :return: A tuple of (boolean, str). If boolean is true, then the stream - """ - stream_availability = self._abstract_availability_strategy.check_availability(logger) - return await stream_availability.is_available(), await stream_availability.message() - - -class AsyncStreamAvailabilityStrategy(AbstractAvailabilityStrategy): - """ - This class acts as an adapter between the existing AvailabilityStrategy and the new AbstractAvailabilityStrategy. - StreamAvailabilityStrategy is instantiated with a Stream and a Source to allow the existing AvailabilityStrategy to be used with the new AbstractAvailabilityStrategy interface. - - A more convenient implementation would not depend on the docs URL instead of the Source itself, and would support running on an AbstractStream instead of only on a Stream. - - This class can be used to help enable concurrency on existing connectors without having to rewrite everything as AbstractStream and AbstractAvailabilityStrategy. - In the long-run, it would be preferable to update the connectors, but we don't have the tooling or need to justify the effort at this time. - """ - - def __init__(self, stream: AsyncStream, source: Source): - """ - :param stream: The stream to delegate to - :param source: The source to delegate to - """ - self._stream = stream - self._source = source - - async def check_availability(self, logger: logging.Logger) -> StreamAvailability: - try: - available, message = await self._stream.check_availability(logger, self._source) - if available: - return StreamAvailable() - else: - return StreamUnavailable(str(message)) - except Exception as e: - display_message = await self._stream.get_error_display_message(e) - if display_message: - raise ExceptionWithDisplayMessage(display_message) - else: - raise e diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py deleted file mode 100644 index 9b48dae873f4..000000000000 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/default_stream_async.py +++ /dev/null @@ -1,80 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -from functools import lru_cache -from logging import Logger -from typing import Any, Iterable, List, Mapping, Optional - -from airbyte_cdk.models import AirbyteStream, SyncMode -from airbyte_cdk.sources.streams.concurrent.abstract_stream_async import AsyncAbstractStream -from airbyte_cdk.sources.streams.concurrent.availability_strategy import AbstractAvailabilityStrategy, StreamAvailability -from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition -from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator - - -class AsyncDefaultStream(AsyncAbstractStream): - def __init__( - self, - partition_generator: PartitionGenerator, - name: str, - json_schema: Mapping[str, Any], - availability_strategy: AbstractAvailabilityStrategy, - primary_key: List[str], - cursor_field: Optional[str], - logger: Logger, - namespace: Optional[str] = None, - ) -> None: - self._stream_partition_generator = partition_generator - self._name = name - self._json_schema = json_schema - self._availability_strategy = availability_strategy - self._primary_key = primary_key - self._cursor_field = cursor_field - self._logger = logger - self._namespace = namespace - - async def generate_partitions(self) -> Iterable[Partition]: - async for partition in self._stream_partition_generator.generate(): - yield partition - - @property - def name(self) -> str: - return self._name - - async def check_availability(self) -> StreamAvailability: - return self._availability_strategy.check_availability(self._logger) - - @property - def cursor_field(self) -> Optional[str]: - return self._cursor_field - - @lru_cache(maxsize=None) - def get_json_schema(self) -> Mapping[str, Any]: - return self._json_schema - - def as_airbyte_stream(self) -> AirbyteStream: - stream = AirbyteStream(name=self.name, json_schema=dict(self._json_schema), supported_sync_modes=[SyncMode.full_refresh]) - - if self._namespace: - stream.namespace = self._namespace - - if self._cursor_field: - stream.source_defined_cursor = True - stream.supported_sync_modes.append(SyncMode.incremental) - stream.default_cursor_field = [self._cursor_field] - - keys = self._primary_key - if keys and len(keys) > 0: - stream.source_defined_primary_key = [keys] - - return stream - - def log_stream_sync_configuration(self) -> None: - self._logger.debug( - f"Syncing stream instance: {self.name}", - extra={ - "primary_key": self._primary_key, - "cursor_field": self.cursor_field, - }, - ) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py deleted file mode 100644 index df7f77dd2411..000000000000 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_async.py +++ /dev/null @@ -1,63 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -from abc import ABC, abstractmethod -from typing import Any, Iterable, Mapping, Optional - -from airbyte_cdk.sources.streams.concurrent.partitions.record import Record - - -class AsyncPartition(ABC): - """ - A partition is responsible for reading a specific set of data from a source. - """ - - @abstractmethod - async def read(self) -> Iterable[Record]: - """ - Reads the data from the partition. - :return: An iterable of records. - """ - pass - - @abstractmethod - def to_slice(self) -> Optional[Mapping[str, Any]]: - """ - Converts the partition to a slice that can be serialized and deserialized. - - Note: it would have been interesting to have a type of `Mapping[str, Comparable]` to simplify typing but some slices can have nested - values ([example](https://github.com/airbytehq/airbyte/blob/1ce84d6396e446e1ac2377362446e3fb94509461/airbyte-integrations/connectors/source-stripe/source_stripe/streams.py#L584-L596)) - :return: A mapping representing a slice - """ - pass - - @abstractmethod - def stream_name(self) -> str: - """ - Returns the name of the stream that this partition is reading from. - :return: The name of the stream. - """ - pass - - @abstractmethod - def close(self) -> None: - """ - Closes the partition. - """ - pass - - @abstractmethod - def is_closed(self) -> bool: - """ - Returns whether the partition is closed. - :return: - """ - pass - - @abstractmethod - def __hash__(self) -> int: - """ - Returns a hash of the partition. - Partitions must be hashable so that they can be used as keys in a dictionary. - """ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py deleted file mode 100644 index d64663d927a1..000000000000 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/concurrent/partitions/partition_generator_async.py +++ /dev/null @@ -1,18 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -from abc import ABC, abstractmethod -from typing import Iterable - -from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition - - -class AsyncPartitionGenerator(ABC): - @abstractmethod - async def generate(self) -> Iterable[Partition]: - """ - Generates partitions for a given sync mode. - :return: An iterable of partitions - """ - pass diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/__init__.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/__init__.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/sources/concurrent_source/async_concurrent_stream_scenarios.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_scenarios.py similarity index 100% rename from airbyte-cdk/python/unit_tests/sources/concurrent_source/async_concurrent_stream_scenarios.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_scenarios.py diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/async_concurrent_stream_source_builder.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_source_builder.py similarity index 100% rename from airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/async_concurrent_stream_source_builder.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_source_builder.py diff --git a/airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/test_async_concurrent_scenarios.py similarity index 100% rename from airbyte-cdk/python/unit_tests/sources/concurrent_source/test_async_concurrent_scenarios.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/test_async_concurrent_scenarios.py diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/__init__.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/__init__.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py similarity index 95% rename from airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py index 71a8fe4e0a23..4a3725a9af8e 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py @@ -8,10 +8,10 @@ import aiohttp import pytest from aioresponses import aioresponses -from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy +from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream from airbyte_cdk.sources.streams import Stream -from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy -from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream logger = logging.getLogger("airbyte") diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py similarity index 99% rename from airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py index 33616db16a1b..f246a210c0e9 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py @@ -14,10 +14,10 @@ import pytest from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import SyncMode +from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream +from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator as HttpTokenAuthenticator -from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream -from airbyte_cdk.sources.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException class StubBasicReadHttpStream(AsyncHttpStream): diff --git a/airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/test_availability_strategy_async.py similarity index 95% rename from airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/streams/test_availability_strategy_async.py index 8c8d93b09eb2..4f62ddbf73fb 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/test_availability_strategy_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/test_availability_strategy_async.py @@ -8,8 +8,8 @@ from airbyte_cdk.models import SyncMode from airbyte_cdk.sources import Source +from airbyte_cdk.sources.async_cdk.streams.availability_strategy_async import AsyncAvailabilityStrategy from airbyte_cdk.sources.streams import Stream -from airbyte_cdk.sources.streams.availability_strategy_async import AsyncAvailabilityStrategy from airbyte_cdk.sources.streams.core import StreamData logger = logging.getLogger("airbyte") diff --git a/airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/test_streams_core_async.py similarity index 98% rename from airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/streams/test_streams_core_async.py index 08bfd1da428b..03d794f1c3a4 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/test_streams_core_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/test_streams_core_async.py @@ -8,7 +8,7 @@ import pytest from airbyte_cdk.models import AirbyteStream, SyncMode -from airbyte_cdk.sources.streams import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream class StreamStubFullRefresh(AsyncStream): diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/__init__.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py similarity index 88% rename from airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py index 09f63f56c63a..bccf5d3e5ae3 100644 --- a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py @@ -4,7 +4,7 @@ import aiohttp import pytest -from airbyte_cdk.sources.streams.http.rate_limiting_async import default_backoff_handler +from airbyte_cdk.sources.async_cdk.streams.http.rate_limiting_async import default_backoff_handler def helper_with_exceptions(exception_type): diff --git a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py similarity index 99% rename from airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py rename to airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py index c82ff0c6a285..4ab673174d54 100644 --- a/airbyte-cdk/python/unit_tests/sources/test_abstract_source_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py @@ -37,10 +37,11 @@ ) from airbyte_cdk.models import Type from airbyte_cdk.models import Type as MessageType -from airbyte_cdk.sources import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import MessageRepository -from airbyte_cdk.sources.streams import AsyncStream, IncrementalMixin +from airbyte_cdk.sources.streams import IncrementalMixin from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message from airbyte_cdk.utils.traced_exception import AirbyteTracedException from pytest import fixture diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py index 988055d78f02..f8492d31588b 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py @@ -6,8 +6,8 @@ import typing from typing import Optional, Tuple +from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from airbyte_cdk.sources.streams import Stream -from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from requests import HTTPError, codes if typing.TYPE_CHECKING: diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index 9ed994f27b9b..2e56fd2a9429 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -11,7 +11,7 @@ from airbyte_cdk import AirbyteLogger from airbyte_cdk.logger import AirbyteLogFormatter from airbyte_cdk.models import AirbyteMessage, AirbyteStateMessage, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, Level, SyncMode -from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import InMemoryMessageRepository from airbyte_cdk.sources.streams import Stream diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index 2038a04cd0c7..f711f78b6812 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -19,10 +19,10 @@ import pandas as pd import pendulum import requests # type: ignore[import] +from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream from airbyte_cdk.models import ConfiguredAirbyteCatalog, FailureType, SyncMode from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy from airbyte_cdk.sources.streams.core import Stream, StreamData -from airbyte_cdk.sources.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream from airbyte_cdk.sources.utils.transform import TransformConfig, TypeTransformer from airbyte_cdk.utils import AirbyteTracedException from numpy import nan diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index 1e394362de8d..cbcef71ab488 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -19,7 +19,7 @@ import requests_mock from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import AirbyteStream, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, DestinationSyncMode, SyncMode, Type -from airbyte_cdk.sources import abstract_source_async +from airbyte_cdk.sources.async_cdk import abstract_source_async from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade from airbyte_cdk.utils import AirbyteTracedException From c5f4908affd56d847b8113e0d7fde89533d619f3 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sat, 6 Jan 2024 19:19:01 -0500 Subject: [PATCH 66/74] __init__.py --- airbyte-cdk/python/airbyte_cdk/sources/async_cdk/__init__.py | 0 .../python/airbyte_cdk/sources/async_cdk/streams/core_async.py | 2 +- .../airbyte_cdk/sources/async_cdk/streams/http/__init__.py | 0 .../airbyte_cdk/sources/async_cdk/streams/utils/__init__.py | 0 4 files changed, 1 insertion(+), 1 deletion(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/__init__.py create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/__init__.py create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/__init__.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py index 442f67ab59d8..9d0ad38483f0 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py @@ -14,7 +14,7 @@ from airbyte_cdk.sources.utils.slice_logger import SliceLogger if TYPE_CHECKING: - from airbyte_cdk.sources.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy + from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy class AsyncStream(Stream, ABC): diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/__init__.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 From cebca8462c45cdf19635addcf5f045138087aeb3 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 7 Jan 2024 00:17:17 -0500 Subject: [PATCH 67/74] Use SourceDispatcher to route requests to async source --- .../async_cdk/abstract_source_async.py | 199 +----------------- .../sources/async_cdk/source_dispatcher.py | 175 +++++++++++++++ .../sources/async_cdk/source_reader.py | 65 ++++++ .../http/availability_strategy_async.py | 3 +- .../async_concurrent_stream_scenarios.py | 3 +- .../async_concurrent_stream_source_builder.py | 20 +- .../test_async_concurrent_scenarios.py | 2 +- .../http/test_availability_strategy_async.py | 17 +- .../async_cdk/test_abstract_source_async.py | 37 ++-- .../sources/scenario_based/helpers.py | 7 + ..._based_concurrent_stream_source_builder.py | 7 +- .../connectors/source-salesforce/main.py | 13 +- .../source_salesforce/__init__.py | 4 +- .../source_salesforce/source.py | 49 +++-- .../source-salesforce/unit_tests/api_test.py | 123 ++++++----- .../source-salesforce/unit_tests/conftest.py | 4 +- .../unit_tests/test_memory.py | 2 +- 17 files changed, 409 insertions(+), 321 deletions(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py index 2c60a36640b9..256e92a61a18 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py @@ -2,15 +2,10 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # -import asyncio import logging -import traceback from abc import ABC, abstractmethod -from queue import Queue -from threading import Lock, Thread -from typing import Any, Callable, Dict, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, TypeVar, Union +from typing import Any, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union -import aiohttp from airbyte_cdk.models import ( AirbyteConnectionStatus, AirbyteMessage, @@ -28,71 +23,8 @@ from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.http import HttpStream from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message -from airbyte_cdk.sources.utils.schema_helpers import InternalConfig, split_config -from airbyte_cdk.utils.event_timing import create_timer +from airbyte_cdk.sources.utils.schema_helpers import InternalConfig from airbyte_cdk.utils.stream_status_utils import as_airbyte_message as stream_status_as_airbyte_message -from airbyte_cdk.utils.traced_exception import AirbyteTracedException - -T = TypeVar("T") -DEFAULT_SESSION_LIMIT = 10000 -DEFAULT_TIMEOUT = None - - -class Sentinel: - def __init__(self, name: str): - self.name = name - - -class SourceReader(Iterator): - def __init__(self, logger: logging.Logger, queue: Queue, sentinels: Dict[str, Sentinel], reader_fn: Callable, *args: Any): - self.logger = logger - self.queue = queue - self.sentinels = sentinels - self.reader_fn = reader_fn - self.reader_args = args - self.sessions: Dict[str, aiohttp.ClientSession] = {} - - self.thread = Thread(target=self._start_reader_thread) - self.thread.start() - - def _start_reader_thread(self): - asyncio.run(self.reader_fn(*self.reader_args)) - - def __next__(self): - loop = asyncio.get_event_loop() - try: - item = self.queue.get(timeout=DEFAULT_TIMEOUT) - if isinstance(item, Exception): - self.logger.error(f"An error occurred in the async thread: {item}") - self.thread.join() - raise item - if isinstance(item, Sentinel): - # Sessions can only be closed once items in the stream have all been dequeued - if session := self.sessions.pop(item.name, None): - loop.create_task(session.close()) # TODO: this can be done better - try: - self.sentinels.pop(item.name) - except KeyError: - raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") - if not self.sentinels: - self.thread.join() - raise StopIteration - else: - return self.__next__() - else: - return item - finally: - loop.create_task(self.cleanup()) - - def drain(self): - while not self.queue.empty(): - yield self.queue.get() - self.thread.join() - - async def cleanup(self): - # pass - for session in self.sessions.values(): - await session.close() class AsyncAbstractSource(AbstractSource, ABC): @@ -101,13 +33,8 @@ class AsyncAbstractSource(AbstractSource, ABC): in this class to create an Airbyte Specification compliant Source. """ - def __init__(self): - super().__init__() - self.queue = Queue(10000) - self.session_limit = DEFAULT_SESSION_LIMIT - @abstractmethod - def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: """ :param logger: source logger :param config: The user-provided configuration as specified by the source's spec. @@ -129,132 +56,28 @@ async def check(self, logger: logging.Logger, config: Mapping[str, Any]) -> Airb return AirbyteConnectionStatus(status=Status.SUCCEEDED) @abstractmethod - def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: + async def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: """ :param config: The user-provided configuration as specified by the source's spec. Any stream construction related operation should happen here. :return: A list of the streams in this source connector. """ - def read( + async def read( self, logger: logging.Logger, config: Mapping[str, Any], catalog: ConfiguredAirbyteCatalog, state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, ) -> Iterator[AirbyteMessage]: - """Implements the Read operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/.""" - logger.info(f"Starting syncing {self.name}") - config, internal_config = split_config(config) - stream_instances = {s.name: s for s in self.streams(config)} - state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) - self._stream_to_instance_map = stream_instances - self._assert_streams(catalog, stream_instances) - - n_records = 0 - with create_timer(self.name) as timer: - for record in self._do_read(catalog, stream_instances, timer, logger, state_manager, internal_config): - n_records += 1 - yield record - - print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") - logger.info(f"Finished syncing {self.name}") - - def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - streams_in_progress_sentinels = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams if s.stream.name in stream_instances} - if not streams_in_progress_sentinels: - return - self.reader = SourceReader(logger, self.queue, streams_in_progress_sentinels, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) - for record in self.reader: - yield record - print("DRAINING __________________________") - for record in self.reader.drain(): - if isinstance(record, Exception): - raise record - yield record - - def _assert_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream]): - for configured_stream in catalog.streams: - stream_instance = stream_instances.get(configured_stream.stream.name) - if not stream_instance: - if not self.raise_exception_on_missing_stream: - return - raise KeyError( - f"The stream {configured_stream.stream.name} no longer exists in the configuration. " - f"Refresh the schema in replication settings and remove this stream from future sync attempts." - ) - - async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - pending_tasks = set() - n_started, n_streams = 0, len(catalog.streams) - streams_iterator = iter(catalog.streams) - exceptions = False - - while (pending_tasks or n_started < n_streams) and not exceptions: - while len(pending_tasks) < self.session_limit and (configured_stream := next(streams_iterator, None)): - if configured_stream is None: - break - # stream_instance = stream_instances.get("Account") # TODO - stream_instance = stream_instances.get(configured_stream.stream.name) - stream = stream_instances.get(configured_stream.stream.name) - self.reader.sessions[configured_stream.stream.name] = await stream.ensure_session() - pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) - n_started += 1 - - done, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) - - for task in done: - if exc := task.exception(): - for remaining_task in pending_tasks: - await remaining_task.cancel() - self.queue.put(exc) - exceptions = True - - async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - try: - await self._async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config) - finally: - self.queue.put(Sentinel(configured_stream.stream.name)) + """ + Implements the Read operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/. - async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - try: - timer.start_event(f"Syncing stream {configured_stream.stream.name}") - stream_is_available, reason = await stream_instance.check_availability(logger, self) - if not stream_is_available: - logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") - return - logger.info(f"Marking stream {configured_stream.stream.name} as STARTED") - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.STARTED)) - async for record in self._read_stream( - logger=logger, - stream_instance=stream_instance, - configured_stream=configured_stream, - state_manager=state_manager, - internal_config=internal_config, - ): - self.queue.put(record) - logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) - except AirbyteTracedException as e: - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) - raise e - except Exception as e: - for message in self._emit_queued_messages(): - self.queue.put(message) - logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") - logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) - display_message = await stream_instance.get_error_display_message(e) - if display_message: - raise AirbyteTracedException.from_exception(e, message=display_message) - else: - raise e - finally: - timer.finish_event() - logger.info(f"Finished syncing {configured_stream.stream.name}") - # logger.info(timer.report()) # TODO - this is causing scenario-based test failures + This method is not used when the AsyncSource is used in conjunction with the AsyncSourceDispatcher. + """ + ... - async def _read_stream( + async def read_stream( self, logger: logging.Logger, stream_instance: AsyncStream, diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py new file mode 100644 index 000000000000..a8fd8bc02217 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py @@ -0,0 +1,175 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import asyncio +import logging +from abc import ABC +from queue import Queue +from typing import Any, Dict, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union + +from airbyte_cdk.models import AirbyteMessage, AirbyteStateMessage, AirbyteStreamStatus, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, ConnectorSpecification +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.abstract_source import AbstractSource +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.source_reader import Sentinel, SourceReader +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream +from airbyte_cdk.sources.utils.schema_helpers import InternalConfig, split_config +from airbyte_cdk.utils.event_timing import create_timer +from airbyte_cdk.utils.traced_exception import AirbyteTracedException +from airbyte_cdk.utils.stream_status_utils import as_airbyte_message as stream_status_as_airbyte_message + +DEFAULT_QUEUE_SIZE = 10000 +DEFAULT_SESSION_LIMIT = 10000 +DEFAULT_TIMEOUT = None + + +class SourceDispatcher(AbstractSource, ABC): + """ + Abstract base class for an Airbyte Source that can dispatch to an async source. + """ + + def __init__(self, async_source: AsyncAbstractSource): + self.async_source = async_source + self.queue = Queue(DEFAULT_QUEUE_SIZE) + self.session_limit = DEFAULT_SESSION_LIMIT + + def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + """ + Run the async_source's `check_connection` method on the event loop. + """ + loop = asyncio.get_event_loop() + return loop.run_until_complete(self.async_source.check_connection(logger, config)) + + def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: + """ + Run the async_source's `streams` method on the event loop. + """ + loop = asyncio.get_event_loop() + return loop.run_until_complete(self.async_source.streams(config)) + + def spec(self, logger: logging.Logger) -> ConnectorSpecification: + """ + Run the async_source's `spec` method. + """ + return self.async_source.spec(logger) + + def read( + self, + logger: logging.Logger, + config: Mapping[str, Any], + catalog: ConfiguredAirbyteCatalog, + state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + ) -> Iterator[AirbyteMessage]: + """ + Run the async_source's `read_streams` method and yield its results. + + """ + logger.info(f"Starting syncing {self.name}") + config, internal_config = split_config(config) + stream_instances = {s.name: s for s in self.streams(config)} + state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) + self._stream_to_instance_map = stream_instances + self._assert_streams(catalog, stream_instances) + + n_records = 0 + with create_timer(self.name) as timer: + for record in self._do_read(catalog, stream_instances, timer, logger, state_manager, internal_config): + n_records += 1 + yield record + + print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") + logger.info(f"Finished syncing {self.name}") + + def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + streams_in_progress_sentinels = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams if s.stream.name in stream_instances} + if not streams_in_progress_sentinels: + return + self.reader = SourceReader(logger, self.queue, streams_in_progress_sentinels, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) + for record in self.reader: + yield record + + for record in self.reader.drain(): + if isinstance(record, Exception): + raise record + yield record + + def _assert_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream]): + for configured_stream in catalog.streams: + stream_instance = stream_instances.get(configured_stream.stream.name) + if not stream_instance: + if not self.async_source.raise_exception_on_missing_stream: + return + raise KeyError( + f"The stream {configured_stream.stream.name} no longer exists in the configuration. " + f"Refresh the schema in replication settings and remove this stream from future sync attempts." + ) + + async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + pending_tasks = set() + n_started, n_streams = 0, len(catalog.streams) + streams_iterator = iter(catalog.streams) + exceptions = False + + while (pending_tasks or n_started < n_streams) and not exceptions: + while len(pending_tasks) < self.session_limit and (configured_stream := next(streams_iterator, None)): + if configured_stream is None: + break + stream_instance = stream_instances.get(configured_stream.stream.name) + stream = stream_instances.get(configured_stream.stream.name) + self.reader.sessions[configured_stream.stream.name] = await stream.ensure_session() + pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) + n_started += 1 + + done, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) + + for task in done: + if exc := task.exception(): + for remaining_task in pending_tasks: + await remaining_task.cancel() + self.queue.put(exc) + exceptions = True + + async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + try: + await self._async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config) + finally: + self.queue.put(Sentinel(configured_stream.stream.name)) + + async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + try: + timer.start_event(f"Syncing stream {configured_stream.stream.name}") + stream_is_available, reason = await stream_instance.check_availability(logger, self) + if not stream_is_available: + logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") + return + logger.info(f"Marking stream {configured_stream.stream.name} as STARTED") + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.STARTED)) + async for record in self.async_source.read_stream( + logger=logger, + stream_instance=stream_instance, + configured_stream=configured_stream, + state_manager=state_manager, + internal_config=internal_config, + ): + self.queue.put(record) + logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) + except AirbyteTracedException as e: + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + raise e + except Exception as e: + for message in self._emit_queued_messages(): + self.queue.put(message) + logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") + logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") + self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + display_message = await stream_instance.get_error_display_message(e) + if display_message: + raise AirbyteTracedException.from_exception(e, message=display_message) + else: + raise e + finally: + timer.finish_event() + logger.info(f"Finished syncing {configured_stream.stream.name}") + # logger.info(timer.report()) # TODO - this is causing scenario-based test failures diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py new file mode 100644 index 000000000000..9a1bc72b86e4 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py @@ -0,0 +1,65 @@ +import asyncio +import logging +from queue import Queue +from threading import Thread +from typing import Any, Callable, Dict, Iterator + +import aiohttp + +DEFAULT_TIMEOUT = None + + +class Sentinel: + def __init__(self, name: str): + self.name = name + + +class SourceReader(Iterator): + def __init__(self, logger: logging.Logger, queue: Queue, sentinels: Dict[str, Sentinel], reader_fn: Callable, *args: Any): + self.logger = logger + self.queue = queue + self.sentinels = sentinels + self.reader_fn = reader_fn + self.reader_args = args + self.sessions: Dict[str, aiohttp.ClientSession] = {} + + self.thread = Thread(target=self._start_reader_thread) + self.thread.start() + + def _start_reader_thread(self): + asyncio.run(self.reader_fn(*self.reader_args)) + + def __next__(self): + loop = asyncio.get_event_loop() + try: + item = self.queue.get(timeout=DEFAULT_TIMEOUT) + if isinstance(item, Exception): + self.logger.error(f"An error occurred in the async thread: {item}") + self.thread.join() + raise item + if isinstance(item, Sentinel): + # Sessions can only be closed once items in the stream have all been dequeued + if session := self.sessions.pop(item.name, None): + loop.create_task(session.close()) # TODO: this can be done better + try: + self.sentinels.pop(item.name) + except KeyError: + raise RuntimeError(f"The sentinel for stream {item.name} was already dequeued. This is unexpected and indicates a possible problem with the connector. Please contact Support.") + if not self.sentinels: + self.thread.join() + raise StopIteration + else: + return self.__next__() + else: + return item + finally: + loop.create_task(self.cleanup()) + + def drain(self): + while not self.queue.empty(): + yield self.queue.get() + self.thread.join() + + async def cleanup(self): + for session in self.sessions.values(): + await session.close() diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py index 722f4bedaead..d024c1fccbb8 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py @@ -8,6 +8,7 @@ import requests from aiohttp import ClientResponseError +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.async_cdk.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy @@ -62,7 +63,7 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, return True, None async def _handle_http_error( - self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: ClientResponseError + self, stream: AsyncStream, logger: logging.Logger, source: Optional["AsyncAbstractSource"], error: ClientResponseError ) -> Tuple[bool, Optional[str]]: """ Override this method to define error handling for various `HTTPError`s diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_scenarios.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_scenarios.py index 09bb29629e3c..cb3fa36a7a8f 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_scenarios.py @@ -1,11 +1,10 @@ # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # -import logging from airbyte_cdk.sources.message import InMemoryMessageRepository from unit_tests.sources.scenario_based.scenario_builder import TestScenarioBuilder -from unit_tests.sources.streams.concurrent.scenarios.async_concurrent_stream_source_builder import ( +from unit_tests.sources.async_cdk.scenario_based.async_concurrent_stream_source_builder import ( AlwaysAvailableAvailabilityStrategy, ConcurrentSourceBuilder, LocalAsyncStream, diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_source_builder.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_source_builder.py index 24604b2c2204..fa21c31ca082 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_source_builder.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/async_concurrent_stream_source_builder.py @@ -2,15 +2,17 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # import logging -from typing import Any, Dict, Iterable, List, Mapping, Optional, Tuple, Union +from typing import Any, Iterable, List, Mapping, Optional, Tuple, Union from airbyte_cdk.models import ConfiguredAirbyteCatalog, ConnectorSpecification, DestinationSyncMode, SyncMode -from airbyte_cdk.sources.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.source_dispatcher import SourceDispatcher +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream +from airbyte_cdk.sources.async_cdk.streams.availability_strategy_async import AsyncAvailabilityStrategy from airbyte_cdk.sources.message import MessageRepository -from airbyte_cdk.sources.streams.core_async import AsyncStream -from airbyte_cdk.sources.streams.availability_strategy_async import AsyncAvailabilityStrategy from airbyte_cdk.sources.streams.core import StreamData from airbyte_protocol.models import ConfiguredAirbyteStream +from unit_tests.sources.scenario_based.helpers import NeverLogSliceLogger from unit_tests.sources.scenario_based.scenario_builder import SourceBuilder @@ -23,7 +25,7 @@ async def check_connection(self, logger: logging.Logger, config: Mapping[str, An # Check is not verified because it is up to the source to implement this method return True, None - def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: + async def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: return self._streams def spec(self, *args: Any, **kwargs: Any) -> ConnectorSpecification: @@ -47,8 +49,12 @@ def __init__(self): self._streams: List[AsyncStream] = [] self._message_repository = None - def build(self, configured_catalog: Optional[Mapping[str, Any]]) -> AsyncConcurrentCdkSource: - return AsyncConcurrentCdkSource(self._streams) + def build(self, configured_catalog: Optional[Mapping[str, Any]]) -> SourceDispatcher: + async_source = AsyncConcurrentCdkSource(self._streams) + async_source._streams = self._streams + async_source._message_repository = self._message_repository + async_source._slice_logger = NeverLogSliceLogger() + return SourceDispatcher(async_source) def set_streams(self, streams: List[AsyncStream]) -> "ConcurrentSourceBuilder": self._streams = streams diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/test_async_concurrent_scenarios.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/test_async_concurrent_scenarios.py index bc7b376b5556..27ceb279f48b 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/test_async_concurrent_scenarios.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/scenario_based/test_async_concurrent_scenarios.py @@ -17,7 +17,7 @@ # test_incremental_stream_without_slice_boundaries_with_concurrent_state, # test_incremental_stream_without_slice_boundaries_with_legacy_state, # ) -from unit_tests.sources.concurrent_source.async_concurrent_stream_scenarios import ( +from unit_tests.sources.async_cdk.scenario_based.async_concurrent_stream_scenarios import ( test_concurrent_cdk_multiple_streams, test_concurrent_cdk_partition_raises_exception, test_concurrent_cdk_single_stream, diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py index 4a3725a9af8e..deb50d2b0c6b 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py @@ -9,6 +9,7 @@ import pytest from aioresponses import aioresponses from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.source_dispatcher import SourceDispatcher from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream from airbyte_cdk.sources.streams import Stream @@ -39,6 +40,7 @@ def retry_factor(self) -> float: return 0.01 +@pytest.mark.asyncio @pytest.mark.parametrize( ("status_code", "expected_is_available", "expected_messages"), [ @@ -59,7 +61,7 @@ def retry_factor(self) -> float: (False, ["Please visit the connector's documentation to learn more."]), ], ) -def test_default_http_availability_strategy( +async def test_default_http_availability_strategy( status_code, expected_is_available, expected_messages, @@ -82,22 +84,21 @@ def __init__(self, streams: List[Stream] = None): async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: return True, "" - def streams(self, config: Mapping[str, Any]) -> List[Stream]: + async def streams(self, config: Mapping[str, Any]) -> List[Stream]: if not self._streams: raise Exception("Stream is not set") return self._streams - loop = asyncio.get_event_loop() - loop.run_until_complete(http_stream.ensure_session()) + await http_stream.ensure_session() with aioresponses() as m: m.get(http_stream.url_base, status=status_code) if include_source: - source = MockSource(streams=[http_stream]) - actual_is_available, reason = loop.run_until_complete(http_stream.check_availability(logger, source)) + source = SourceDispatcher(MockSource(streams=[http_stream])) + actual_is_available, reason = await http_stream.check_availability(logger, source.async_source) else: - actual_is_available, reason = loop.run_until_complete(http_stream.check_availability(logger)) + actual_is_available, reason = await http_stream.check_availability(logger) assert expected_is_available == actual_is_available if expected_is_available: @@ -107,7 +108,7 @@ def streams(self, config: Mapping[str, Any]) -> List[Stream]: for message in all_expected_messages: assert message in reason - loop.run_until_complete(http_stream._session.close()) + await http_stream._session.close() def test_http_availability_raises_unhandled_error(mocker): diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py index 4ab673174d54..f0842460d480 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py @@ -38,6 +38,7 @@ from airbyte_cdk.models import Type from airbyte_cdk.models import Type as MessageType from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.source_dispatcher import SourceDispatcher from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import MessageRepository @@ -70,7 +71,7 @@ async def check_connection(self, logger: logging.Logger, config: Mapping[str, An return self.check_lambda() return False, "Missing callable." - def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: + async def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: if not self._streams: raise Exception("Stream is not set") return self._streams @@ -242,7 +243,7 @@ def test_discover(mocker): mocker.patch.object(stream2, "as_airbyte_stream", return_value=airbyte_stream2) expected = AirbyteCatalog(streams=[airbyte_stream1, airbyte_stream2]) - src = MockSource(check_lambda=lambda: (True, None), streams=[stream1, stream2]) + src = SourceDispatcher(MockSource(check_lambda=lambda: (True, None), streams=[stream1, stream2])) assert expected == src.discover(logger, {}) @@ -254,7 +255,7 @@ def test_read_nonexistent_stream_raises_exception(mocker): mocker.patch.object(MockStream, "get_json_schema", return_value={}) - src = MockSource(streams=[s1]) + src = SourceDispatcher(MockSource(streams=[s1])) catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(s2, SyncMode.full_refresh)]) with pytest.raises(KeyError): list(src.read(logger, {}, catalog)) @@ -267,7 +268,7 @@ def test_read_nonexistent_stream_without_raises_exception(mocker): mocker.patch.object(MockStream, "get_json_schema", return_value={}) - src = MockSource(streams=[s1], exception_on_missing_stream=False) + src = SourceDispatcher(MockSource(streams=[s1], exception_on_missing_stream=False)) catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(s2, SyncMode.full_refresh)]) messages = list(src.read(logger, {}, catalog)) @@ -286,7 +287,7 @@ def test_read_stream_emits_repository_message_before_record(mocker, message_repo mocker.patch.object(MockStream, "read_records", fake_read_records) message_repository.consume_queue.side_effect = [[message for message in [MESSAGE_FROM_REPOSITORY]], []] - source = MockSource(streams=[stream], message_repository=message_repository) + source = SourceDispatcher(MockSource(streams=[stream], message_repository=message_repository)) messages = list(source.read(logger, {}, ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]))) @@ -301,7 +302,7 @@ def test_read_stream_emits_repository_message_on_error(mocker, message_repositor mocker.patch.object(MockStream, "read_records", side_effect=RuntimeError("error")) message_repository.consume_queue.return_value = [message for message in [MESSAGE_FROM_REPOSITORY]] - source = MockSource(streams=[stream], message_repository=message_repository) + source = SourceDispatcher(MockSource(streams=[stream], message_repository=message_repository)) with pytest.raises(RuntimeError): messages = list(source.read(logger, {}, ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]))) assert MESSAGE_FROM_REPOSITORY in messages @@ -318,7 +319,7 @@ def test_read_stream_with_error_gets_display_message(mocker): mocker.patch.object(MockStream, "get_json_schema", return_value={}) stream.read_records = read_records_with_error - source = MockSource(streams=[stream]) + source = SourceDispatcher(MockSource(streams=[stream])) catalog = ConfiguredAirbyteCatalog(streams=[_configured_stream(stream, SyncMode.full_refresh)]) # without get_error_display_message @@ -399,7 +400,7 @@ def test_valid_full_refresh_read_no_slices(mocker): mocker.patch.object(MockStream, "get_json_schema", return_value={}) - src = MockSource(streams=[s1, s2]) + src = SourceDispatcher(MockSource(streams=[s1, s2])) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(s1, SyncMode.full_refresh), @@ -444,7 +445,7 @@ async def _fake_stream_slices(*args, **kwargs): mocker.patch.object(MockStream, "get_json_schema", return_value={}) mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) - src = MockSource(streams=[s1, s2]) + src = SourceDispatcher(MockSource(streams=[s1, s2])) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(s1, SyncMode.full_refresh), @@ -495,7 +496,7 @@ async def _fake_stream_slices(*args, **kwargs): mocker.patch.object(MockStream, "get_json_schema", return_value={}) mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) - src = MockSource(streams=[stream]) + src = SourceDispatcher(MockSource(streams=[stream])) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream, SyncMode.full_refresh), @@ -524,7 +525,7 @@ async def _fake_stream_slices(*args, **kwargs): mocker.patch.object(MockStream, "get_json_schema", return_value={}) mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) - src = MockSource(streams=[stream]) + src = SourceDispatcher(MockSource(streams=[stream])) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream, SyncMode.incremental), @@ -589,7 +590,7 @@ def test_with_state_attribute(self, mocker, use_legacy, per_stream_enabled): return_value=new_state_from_connector, ) mocker.patch.object(MockStreamWithState, "get_json_schema", return_value={}) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), @@ -670,7 +671,7 @@ def test_with_checkpoint_interval(self, mocker, use_legacy, per_stream_enabled): return_value=1, ) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), @@ -739,7 +740,7 @@ def test_with_no_interval(self, mocker, use_legacy, per_stream_enabled): mocker.patch.object(MockStream, "supports_incremental", return_value=True) mocker.patch.object(MockStream, "get_json_schema", return_value={}) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), @@ -828,7 +829,7 @@ async def _fake_stream_slices(*args, **kwargs): mocker.patch.object(MockStream, "get_json_schema", return_value={}) mocker.patch.object(MockStream, "stream_slices", _fake_stream_slices) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), @@ -935,7 +936,7 @@ async def _fake_stream_slices(*args, **kwargs): return_value=2, ) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), @@ -1029,7 +1030,7 @@ async def _fake_stream_slices(*args, **kwargs): return_value=2, ) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), @@ -1146,7 +1147,7 @@ async def _fake_stream_slices(*args, **kwargs): return_value=2, ) - src = MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled) + src = SourceDispatcher(MockSource(streams=[stream_1, stream_2], per_stream=per_stream_enabled)) catalog = ConfiguredAirbyteCatalog( streams=[ _configured_stream(stream_1, SyncMode.incremental), diff --git a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py index 5c664ad01292..0e19cec00c6e 100644 --- a/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py +++ b/airbyte-cdk/python/unit_tests/sources/scenario_based/helpers.py @@ -3,6 +3,7 @@ # import json +import logging import math from pathlib import Path, PosixPath from typing import Any, Dict, List, Mapping, Optional, Union @@ -12,6 +13,7 @@ from airbyte_cdk.entrypoint import get_source_iter from airbyte_cdk.models import AirbyteAnalyticsTraceMessage, SyncMode from airbyte_cdk.sources import AbstractSource +from airbyte_cdk.sources.utils.slice_logger import SliceLogger from airbyte_cdk.test.entrypoint_wrapper import EntrypointOutput from airbyte_cdk.test.entrypoint_wrapper import read as entrypoint_read from airbyte_cdk.utils.traced_exception import AirbyteTracedException @@ -190,3 +192,8 @@ def get_error_message_from_exc(exc: ExceptionInfo[Any]) -> str: if isinstance(exc.value, AirbyteTracedException): return exc.value.message return str(exc.value.args[0]) + + +class NeverLogSliceLogger(SliceLogger): + def should_log_slice_message(self, logger: logging.Logger) -> bool: + return False diff --git a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py index 45290ff73ba2..db29bae41516 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/concurrent/scenarios/thread_based_concurrent_stream_source_builder.py @@ -18,8 +18,8 @@ from airbyte_cdk.sources.streams.concurrent.partitions.partition_generator import PartitionGenerator from airbyte_cdk.sources.streams.concurrent.partitions.record import Record from airbyte_cdk.sources.streams.core import StreamData -from airbyte_cdk.sources.utils.slice_logger import SliceLogger from airbyte_protocol.models import ConfiguredAirbyteStream +from unit_tests.sources.scenario_based.helpers import NeverLogSliceLogger from unit_tests.sources.scenario_based.scenario_builder import SourceBuilder @@ -134,8 +134,3 @@ def set_message_repository(self, message_repository: MessageRepository) -> "Conc class AlwaysAvailableAvailabilityStrategy(AbstractAvailabilityStrategy): def check_availability(self, logger: logging.Logger) -> StreamAvailability: return StreamAvailable() - - -class NeverLogSliceLogger(SliceLogger): - def should_log_slice_message(self, logger: logging.Logger) -> bool: - return False diff --git a/airbyte-integrations/connectors/source-salesforce/main.py b/airbyte-integrations/connectors/source-salesforce/main.py index d871182bd494..7643155bde89 100644 --- a/airbyte-integrations/connectors/source-salesforce/main.py +++ b/airbyte-integrations/connectors/source-salesforce/main.py @@ -10,16 +10,18 @@ from airbyte_cdk.entrypoint import AirbyteEntrypoint, launch from airbyte_cdk.models import AirbyteErrorTraceMessage, AirbyteMessage, AirbyteTraceMessage, TraceType, Type -from source_salesforce import SourceSalesforce +from source_salesforce import SalesforceSourceDispatcher, SourceSalesforce def _get_source(args: List[str]): catalog_path = AirbyteEntrypoint.extract_catalog(args) config_path = AirbyteEntrypoint.extract_config(args) try: - return SourceSalesforce( - SourceSalesforce.read_catalog(catalog_path) if catalog_path else None, - SourceSalesforce.read_config(config_path) if config_path else None, + return SalesforceSourceDispatcher( + SourceSalesforce( + SourceSalesforce.read_catalog(catalog_path) if catalog_path else None, + SourceSalesforce.read_config(config_path) if config_path else None, + ) ) except Exception as error: print( @@ -39,7 +41,8 @@ def _get_source(args: List[str]): if __name__ == "__main__": - _args = ['read', '--config', 'secrets/config.json', '--catalog', 'integration_tests/configured_catalog.json'] + # _args = ['read', '--config', 'secrets/config.json', '--catalog', 'integration_tests/configured_catalog.json'] + _args = sys.argv[1:] source = _get_source(_args) if source: launch(source, _args) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py index 60d17cf1e8f0..fb3d4279e4ec 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py @@ -2,6 +2,6 @@ # Copyright (c) 2021 Airbyte, Inc., all rights reserved. # -from .source import SourceSalesforce +from .source import SourceSalesforce, SalesforceSourceDispatcher -__all__ = ["SourceSalesforce"] +__all__ = ["SourceSalesforce", "SalesforceSourceDispatcher"] diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index 2e56fd2a9429..706a051ea99c 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -12,6 +12,7 @@ from airbyte_cdk.logger import AirbyteLogFormatter from airbyte_cdk.models import AirbyteMessage, AirbyteStateMessage, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, Level, SyncMode from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.source_dispatcher import SourceDispatcher from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.message import InMemoryMessageRepository from airbyte_cdk.sources.streams import Stream @@ -58,14 +59,14 @@ def __init__(self, catalog: Optional[ConfiguredAirbyteCatalog], config: Optional super().__init__() @staticmethod - def _get_sf_object(config: Mapping[str, Any]) -> Salesforce: + async def _get_sf_object(config: Mapping[str, Any]) -> Salesforce: sf = Salesforce(**config) sf.login() return sf - def check_connection(self, logger: AirbyteLogger, config: Mapping[str, Any]) -> Tuple[bool, Optional[str]]: + async def check_connection(self, logger: AirbyteLogger, config: Mapping[str, Any]) -> Tuple[bool, Optional[str]]: try: - salesforce = self._get_sf_object(config) + salesforce = await self._get_sf_object(config) salesforce.describe() except exceptions.HTTPError as error: error_msg = f"An error occurred: {error.response.text}" @@ -145,7 +146,7 @@ def prepare_stream(cls, stream_name: str, json_schema, sobject_options, sf_objec return stream_class, stream_kwargs @classmethod - def generate_streams( + async def generate_streams( cls, config: Mapping[str, Any], stream_objects: Mapping[str, Any], @@ -180,12 +181,12 @@ def generate_streams( streams.append(stream) return streams - def streams(self, config: Mapping[str, Any]) -> List[Stream]: + async def streams(self, config: Mapping[str, Any]) -> List[Stream]: if not config.get("start_date"): config["start_date"] = (datetime.now() - relativedelta(years=self.START_DATE_OFFSET_IN_YEARS)).strftime(self.DATETIME_FORMAT) - sf = self._get_sf_object(config) + sf = await self._get_sf_object(config) stream_objects = sf.get_validated_streams(config=config, catalog=self.catalog) - streams = self.generate_streams(config, stream_objects, sf) + streams = await self.generate_streams(config, stream_objects, sf) streams.append(Describe(sf_api=sf, catalog=self.catalog)) # TODO: incorporate state & ConcurrentCursor when we support incremental configured_streams = [] @@ -200,21 +201,7 @@ def _get_sync_mode_from_catalog(self, stream: Stream) -> Optional[SyncMode]: return catalog_stream.sync_mode return None - def read( - self, - logger: logging.Logger, - config: Mapping[str, Any], - catalog: ConfiguredAirbyteCatalog, - state: Union[List[AirbyteStateMessage], MutableMapping[str, Any]] = None, - ) -> Iterator[AirbyteMessage]: - # save for use inside streams method - self.catalog = catalog - try: - yield from super().read(logger, config, catalog, state) - except AirbyteStopSync: - logger.info(f"Finished syncing {self.name}") - - async def _read_stream( + async def read_stream( self, logger: logging.Logger, stream_instance: Stream, @@ -223,7 +210,7 @@ async def _read_stream( internal_config: InternalConfig, ) -> Iterator[AirbyteMessage]: try: - async for record in super()._read_stream(logger, stream_instance, configured_stream, state_manager, internal_config): + async for record in super().read_stream(logger, stream_instance, configured_stream, state_manager, internal_config): yield record except aiohttp.ClientResponseError as error: error_data = error._response_error if hasattr(error, "_response_error") else {} @@ -233,3 +220,19 @@ async def _read_stream( logger.warning(f"API Call {url} limit is exceeded. Error message: '{error_data.get('message')}'") raise AirbyteStopSync() # if got 403 rate limit response, finish the sync with success. raise error + + +class SalesforceSourceDispatcher(SourceDispatcher): + def read( + self, + logger: logging.Logger, + config: Mapping[str, Any], + catalog: ConfiguredAirbyteCatalog, + state: Union[List[AirbyteStateMessage], MutableMapping[str, Any]] = None, + ) -> Iterator[AirbyteMessage]: + # save for use inside streams method + self.catalog = catalog + try: + yield from super().read(logger, config, catalog, state) + except AirbyteStopSync: + logger.info(f"Finished syncing {self.async_source.name}") diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index cbcef71ab488..43084f71f77f 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -1,8 +1,7 @@ # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # - - +import asyncio import csv import io import logging @@ -19,14 +18,14 @@ import requests_mock from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import AirbyteStream, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, DestinationSyncMode, SyncMode, Type -from airbyte_cdk.sources.async_cdk import abstract_source_async +from airbyte_cdk.sources.async_cdk import source_dispatcher from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade from airbyte_cdk.utils import AirbyteTracedException from conftest import encoding_symbols_parameters, generate_stream from source_salesforce.api import Salesforce from source_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING -from source_salesforce.source import SourceSalesforce +from source_salesforce.source import SalesforceSourceDispatcher, SourceSalesforce from source_salesforce.streams import ( CSV_FIELD_SIZE_LIMIT, BulkIncrementalSalesforceStream, @@ -66,7 +65,7 @@ def test_login_authentication_error_handler( stream_config, requests_mock, login_status_code, login_json_resp, expected_error_msg, is_config_error ): - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) logger = logging.getLogger("airbyte") requests_mock.register_uri( "POST", "https://login.salesforce.com/services/oauth2/token", json=login_json_resp, status_code=login_status_code @@ -84,7 +83,7 @@ def test_login_authentication_error_handler( @pytest.mark.asyncio async def test_bulk_sync_creation_failed(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() def callback(*args, **kwargs): @@ -106,7 +105,7 @@ async def test_bulk_stream_fallback_to_rest(stream_config, stream_api): On the other hand, we mock REST API for this same entity with a successful response. After having instantiated a BulkStream, sync should succeed in case it falls back to REST API. Otherwise it would throw an error. """ - stream = generate_stream("CustomEntity", stream_config, stream_api) + stream = await generate_stream("CustomEntity", stream_config, stream_api) await stream.ensure_session() def callback(*args, **kwargs): @@ -133,28 +132,30 @@ async def get_records(*args, **kwargs): await stream._session.close() -def test_stream_unsupported_by_bulk(stream_config, stream_api): +@pytest.mark.asyncio +async def test_stream_unsupported_by_bulk(stream_config, stream_api): """ Stream `AcceptedEventRelation` is not supported by BULK API, so that REST API stream will be used for it. """ stream_name = "AcceptedEventRelation" - stream = generate_stream(stream_name, stream_config, stream_api) + stream = await generate_stream(stream_name, stream_config, stream_api) assert not isinstance(stream, BulkSalesforceStream) -def test_stream_contains_unsupported_properties_by_bulk(stream_config, stream_api_v2): +@pytest.mark.asyncio +async def test_stream_contains_unsupported_properties_by_bulk(stream_config, stream_api_v2): """ Stream `Account` contains compound field such as BillingAddress, which is not supported by BULK API (csv), in that case REST API stream will be used for it. """ stream_name = "Account" - stream = generate_stream(stream_name, stream_config, stream_api_v2) + stream = await generate_stream(stream_name, stream_config, stream_api_v2) assert not isinstance(stream, BulkSalesforceStream) @pytest.mark.asyncio async def test_bulk_sync_pagination(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() job_id = "fake_job" call_counter = 0 @@ -211,7 +212,7 @@ async def _get_result_id(stream): @pytest.mark.asyncio async def test_bulk_sync_successful(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -226,7 +227,7 @@ async def test_bulk_sync_successful(stream_config, stream_api): @pytest.mark.asyncio async def test_bulk_sync_successful_long_response(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -243,7 +244,7 @@ async def test_bulk_sync_successful_long_response(stream_config, stream_api): @pytest.mark.asyncio @pytest.mark.timeout(17) async def test_bulk_sync_successful_retry(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -264,7 +265,7 @@ async def test_bulk_sync_successful_retry(stream_config, stream_api): @pytest.mark.asyncio @pytest.mark.timeout(30) async def test_bulk_sync_failed_retry(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -280,6 +281,8 @@ async def test_bulk_sync_failed_retry(stream_config, stream_api): await stream._session.close() + +@pytest.mark.asyncio @pytest.mark.parametrize( "start_date_provided,stream_name,expected_start_date", [ @@ -287,7 +290,7 @@ async def test_bulk_sync_failed_retry(stream_config, stream_api): (True, "ActiveFeatureLicenseMetric", "2010-01-18T21:18:20Z"), ], ) -def test_stream_start_date( +async def test_stream_start_date( start_date_provided, stream_name, expected_start_date, @@ -296,27 +299,29 @@ def test_stream_start_date( stream_config_without_start_date, ): if start_date_provided: - stream = generate_stream(stream_name, stream_config, stream_api) + stream = await generate_stream(stream_name, stream_config, stream_api) assert stream.start_date == expected_start_date else: - stream = generate_stream(stream_name, stream_config_without_start_date, stream_api) + stream = await generate_stream(stream_name, stream_config_without_start_date, stream_api) assert datetime.strptime(stream.start_date, "%Y-%m-%dT%H:%M:%SZ").year == datetime.now().year - 2 -def test_stream_start_date_should_be_converted_to_datetime_format(stream_config_date_format, stream_api): - stream: IncrementalRestSalesforceStream = generate_stream("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) +@pytest.mark.asyncio +async def test_stream_start_date_should_be_converted_to_datetime_format(stream_config_date_format, stream_api): + stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) assert stream.start_date == "2010-01-18T00:00:00Z" -def test_stream_start_datetime_format_should_not_changed(stream_config, stream_api): - stream: IncrementalRestSalesforceStream = generate_stream("ActiveFeatureLicenseMetric", stream_config, stream_api) +@pytest.mark.asyncio +async def test_stream_start_datetime_format_should_not_changed(stream_config, stream_api): + stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config, stream_api) assert stream.start_date == "2010-01-18T21:18:20Z" @pytest.mark.asyncio async def test_download_data_filter_null_bytes(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -334,7 +339,7 @@ async def test_download_data_filter_null_bytes(stream_config, stream_api): @pytest.mark.asyncio async def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -347,7 +352,7 @@ async def test_read_with_chunks_should_return_only_object_data_type(stream_confi @pytest.mark.asyncio async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -360,7 +365,7 @@ async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_d @pytest.mark.asyncio async def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -378,7 +383,7 @@ async def test_read_with_chunks_should_return_null_value_when_no_data_is_provide ) async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -425,32 +430,32 @@ def configure_request_params_mock(stream_1, stream_2): stream_2.request_params.return_value = {"q": "query"} -@pytest.mark.asyncio -async def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): +def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): """ Connector should stop the sync if one stream reached rate limit stream_1, stream_2, stream_3, ... While reading `stream_1` if 403 (Rate Limit) is received, it should finish that stream with success and stop the sync process. Next streams should not be executed. """ - abstract_source_async.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time + source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time stream_config.update({"start_date": "2021-10-01"}) - stream_1: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - stream_2: BulkIncrementalSalesforceStream = generate_stream("Asset", stream_config, stream_api) + loop = asyncio.get_event_loop() + stream_1: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Account", stream_config, stream_api)) + stream_2: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Asset", stream_config, stream_api)) streams = [stream_1, stream_2] configure_request_params_mock(stream_1, stream_2) stream_1.page_size = 6 stream_1.state_checkpoint_interval = 5 - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = streams logger = logging.getLogger("airbyte") json_response = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} - orig_read_stream = source._read_stream + orig_read_stream = source.async_source.read_stream async def patched_read_stream(*args, **kwargs): base_url = f"{stream_1.sf_api.instance_url}{stream_1.path()}" @@ -482,7 +487,7 @@ def cb(response): async for r in orig_read_stream(**kwargs): yield r - source._read_stream = patched_read_stream + source.async_source.read_stream = patched_read_stream result = [i for i in source.read(logger=logger, config=stream_config, catalog=bulk_catalog, state=state)] assert stream_1.request_params.called @@ -499,14 +504,14 @@ def cb(response): @pytest.mark.asyncio async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): - abstract_source_async.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time + source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time stream_config.update({"start_date": "2021-11-01"}) - stream_1: IncrementalRestSalesforceStream = generate_stream("KnowledgeArticle", stream_config, stream_api) - stream_2: IncrementalRestSalesforceStream = generate_stream("AcceptedEventRelation", stream_config, stream_api) + stream_1: IncrementalRestSalesforceStream = await generate_stream("KnowledgeArticle", stream_config, stream_api) + stream_2: IncrementalRestSalesforceStream = await generate_stream("AcceptedEventRelation", stream_config, stream_api) stream_1.state_checkpoint_interval = 3 configure_request_params_mock(stream_1, stream_2) - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = [stream_1, stream_2] @@ -591,7 +596,7 @@ async def check_availability(*args, **kwargs): @pytest.mark.asyncio async def test_pagination_rest(stream_config, stream_api): stream_name = "AcceptedEventRelation" - stream: RestSalesforceStream = generate_stream(stream_name, stream_config, stream_api) + stream: RestSalesforceStream = await generate_stream(stream_name, stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds next_page_url = "/services/data/v57.0/query/012345" await stream.ensure_session() @@ -807,22 +812,24 @@ def test_csv_field_size_limit(): pass -def test_convert_to_standard_instance(stream_config, stream_api): - bulk_stream = generate_stream("Account", stream_config, stream_api) +@pytest.mark.asyncio +async def test_convert_to_standard_instance(stream_config, stream_api): + bulk_stream = await generate_stream("Account", stream_config, stream_api) rest_stream = bulk_stream.get_standard_instance() assert isinstance(rest_stream, IncrementalRestSalesforceStream) -def test_rest_stream_init_with_too_many_properties(stream_config, stream_api_v2_too_many_properties): +@pytest.mark.asyncio +async def test_rest_stream_init_with_too_many_properties(stream_config, stream_api_v2_too_many_properties): with pytest.raises(AssertionError): # v2 means the stream is going to be a REST stream. # A missing primary key is not allowed - generate_stream("Account", stream_config, stream_api_v2_too_many_properties) + await generate_stream("Account", stream_config, stream_api_v2_too_many_properties) @pytest.mark.asyncio async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): - stream = generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) await stream.ensure_session() chunks = list(stream.chunk_properties()) for chunk in chunks: @@ -864,7 +871,7 @@ async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_prop @pytest.mark.asyncio async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties): - stream = generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) chunks = list(stream.chunk_properties()) for chunk in chunks: assert stream.primary_key in chunk @@ -899,7 +906,7 @@ async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_p ], ) async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api, status_code, response_json, log_message, caplog): - stream = generate_stream("Account", stream_config, stream_api) + stream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query" @@ -928,7 +935,7 @@ async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api ], ) async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, status_code, response_json, error_message): - stream = generate_stream("Account", stream_config, stream_api) + stream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query/queryJobId" @@ -943,7 +950,7 @@ async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, stat @pytest.mark.asyncio() @freezegun.freeze_time("2023-01-01") async def test_bulk_stream_slices(stream_config_date_format, stream_api): - stream: BulkIncrementalSalesforceStream = generate_stream("FakeBulkStream", stream_config_date_format, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("FakeBulkStream", stream_config_date_format, stream_api) stream_slices = [s async for s in stream.stream_slices(sync_mode=SyncMode.full_refresh)] expected_slices = [] today = pendulum.today(tz="UTC") @@ -963,10 +970,10 @@ async def test_bulk_stream_slices(stream_config_date_format, stream_api): @freezegun.freeze_time("2023-04-01") async def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog): stream_config_date_format.update({"start_date": "2023-01-01"}) - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config_date_format, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config_date_format, stream_api) await stream.ensure_session() - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = [stream] base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -1024,15 +1031,17 @@ async def test_bulk_stream_request_params_states(stream_config_date_format, stre assert actual_state_values == expected_state_values -def test_request_params_incremental(stream_config_date_format, stream_api): - stream = generate_stream("ContentDocument", stream_config_date_format, stream_api) +@pytest.mark.asyncio +async def test_request_params_incremental(stream_config_date_format, stream_api): + stream = await generate_stream("ContentDocument", stream_config_date_format, stream_api) params = stream.request_params(stream_state={}, stream_slice={'start_date': '2020', 'end_date': '2021'}) assert params == {'q': 'SELECT LastModifiedDate, Id FROM ContentDocument WHERE LastModifiedDate >= 2020 AND LastModifiedDate < 2021'} -def test_request_params_substream(stream_config_date_format, stream_api): - stream = generate_stream("ContentDocumentLink", stream_config_date_format, stream_api) +@pytest.mark.asyncio +async def test_request_params_substream(stream_config_date_format, stream_api): + stream = await generate_stream("ContentDocumentLink", stream_config_date_format, stream_api) params = stream.request_params(stream_state={}, stream_slice={'parents': [{'Id': 1}, {'Id': 2}]}) assert params == {"q": "SELECT LastModifiedDate, Id FROM ContentDocumentLink WHERE ContentDocumentId IN ('1','2')"} @@ -1042,7 +1051,7 @@ def test_request_params_substream(stream_config_date_format, stream_api): @freezegun.freeze_time("2023-03-20") async def test_stream_slices_for_substream(stream_config, stream_api): stream_config['start_date'] = '2023-01-01' - stream: BulkSalesforceSubStream = generate_stream("ContentDocumentLink", stream_config, stream_api) + stream: BulkSalesforceSubStream = await generate_stream("ContentDocumentLink", stream_config, stream_api) stream.SLICE_BATCH_SIZE = 2 # each ContentDocumentLink should contain 2 records from parent ContentDocument stream await stream.ensure_session() diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py index eeacdd2235d2..b855715b5a1f 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py @@ -119,8 +119,8 @@ def stream_api_v2_pk_too_many_properties(stream_config): return _stream_api(stream_config, describe_response_data=describe_response_data) -def generate_stream(stream_name, stream_config, stream_api): - return SourceSalesforce.generate_streams(stream_config, {stream_name: None}, stream_api)[0] +async def generate_stream(stream_name, stream_config, stream_api): + return (await SourceSalesforce.generate_streams(stream_config, {stream_name: None}, stream_api))[0] def encoding_symbols_parameters(): diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py index 6e38811903bf..bbf2e230127a 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py @@ -29,7 +29,7 @@ @pytest.mark.asyncio async def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) await stream.ensure_session() content = b'"Id","IsDeleted"' for _ in range(n_records): From 5874671fa18da8b27a042ad37414bca9aeb36265 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 7 Jan 2024 15:03:08 -0500 Subject: [PATCH 68/74] Fix types and formatting --- .../airbyte_cdk/sources/abstract_source.py | 135 +++++++-- .../async_cdk/abstract_source_async.py | 73 +++-- .../sources/async_cdk/source_dispatcher.py | 184 ++++++++++-- .../async_cdk/streams/async_call_rate.py | 20 +- .../streams/availability_strategy_async.py | 14 +- .../sources/async_cdk/streams/core_async.py | 83 ++++-- .../http/availability_strategy_async.py | 49 +++- .../async_cdk/streams/http/http_async.py | 268 ++++++++++++++---- .../streams/http/rate_limiting_async.py | 29 +- .../streams/utils/stream_helper_async.py | 22 +- .../airbyte_cdk/sources/streams/http/http.py | 174 +++++++++--- .../sources/streams/http/http_base.py | 25 +- 12 files changed, 841 insertions(+), 235 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source.py b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source.py index 0f8bf716cc10..854bdff11a74 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/abstract_source.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/abstract_source.py @@ -4,7 +4,17 @@ import logging from abc import ABC, abstractmethod -from typing import Any, Dict, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union +from typing import ( + Any, + Iterable, + Iterator, + List, + Mapping, + MutableMapping, + Optional, + Tuple, + Union, +) from airbyte_cdk.models import ( AirbyteCatalog, @@ -28,7 +38,9 @@ from airbyte_cdk.sources.utils.schema_helpers import InternalConfig, split_config from airbyte_cdk.sources.utils.slice_logger import DebugSliceLogger, SliceLogger from airbyte_cdk.utils.event_timing import create_timer -from airbyte_cdk.utils.stream_status_utils import as_airbyte_message as stream_status_as_airbyte_message +from airbyte_cdk.utils.stream_status_utils import ( + as_airbyte_message as stream_status_as_airbyte_message, +) from airbyte_cdk.utils.traced_exception import AirbyteTracedException _default_message_repository = InMemoryMessageRepository() @@ -41,7 +53,9 @@ class AbstractSource(Source, ABC): """ @abstractmethod - def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + def check_connection( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> Tuple[bool, Optional[Any]]: """ :param logger: source logger :param config: The user-provided configuration as specified by the source's spec. @@ -62,7 +76,7 @@ def streams(self, config: Mapping[str, Any]) -> List[Stream]: """ # Stream name to instance map for applying output object transformation - _stream_to_instance_map: Dict[str, Stream] = {} + _stream_to_instance_map: Mapping[str, Stream] = {} _slice_logger: SliceLogger = DebugSliceLogger() @property @@ -70,14 +84,18 @@ def name(self) -> str: """Source name""" return self.__class__.__name__ - def discover(self, logger: logging.Logger, config: Mapping[str, Any]) -> AirbyteCatalog: + def discover( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> AirbyteCatalog: """Implements the Discover operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/#discover. """ streams = [stream.as_airbyte_stream() for stream in self.streams(config=config)] return AirbyteCatalog(streams=streams) - def check(self, logger: logging.Logger, config: Mapping[str, Any]) -> AirbyteConnectionStatus: + def check( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> AirbyteConnectionStatus: """Implements the Check Connection operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/#check. """ @@ -91,7 +109,9 @@ def read( logger: logging.Logger, config: Mapping[str, Any], catalog: ConfiguredAirbyteCatalog, - state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + state: Optional[ + Union[List[AirbyteStateMessage], MutableMapping[str, Any]] + ] = None, ) -> Iterator[AirbyteMessage]: """Implements the Read operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/.""" logger.info(f"Starting syncing {self.name}") @@ -99,7 +119,9 @@ def read( # TODO assert all streams exist in the connector # get the streams once in case the connector needs to make any queries to generate them stream_instances = {s.name: s for s in self.streams(config)} - state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) + state_manager = ConnectorStateManager( + stream_instance_map=stream_instances, state=state + ) self._stream_to_instance_map = stream_instances stream_name_to_exception: MutableMapping[str, AirbyteTracedException] = {} @@ -117,12 +139,20 @@ def read( try: timer.start_event(f"Syncing stream {configured_stream.stream.name}") - stream_is_available, reason = stream_instance.check_availability(logger, self) + stream_is_available, reason = stream_instance.check_availability( + logger, self + ) if not stream_is_available: - logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") + logger.warning( + f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}" + ) continue - logger.info(f"Marking stream {configured_stream.stream.name} as STARTED") - yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.STARTED) + logger.info( + f"Marking stream {configured_stream.stream.name} as STARTED" + ) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.STARTED + ) yield from self._read_stream( logger=logger, stream_instance=stream_instance, @@ -130,22 +160,36 @@ def read( state_manager=state_manager, internal_config=internal_config, ) - logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") - yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE) + logger.info( + f"Marking stream {configured_stream.stream.name} as STOPPED" + ) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.COMPLETE + ) except AirbyteTracedException as e: - yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.INCOMPLETE + ) if self.continue_sync_on_stream_failure: stream_name_to_exception[stream_instance.name] = e else: raise e except Exception as e: yield from self._emit_queued_messages() - logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") - logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") - yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE) + logger.exception( + f"Encountered an exception while reading stream {configured_stream.stream.name}" + ) + logger.info( + f"Marking stream {configured_stream.stream.name} as STOPPED" + ) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.INCOMPLETE + ) display_message = stream_instance.get_error_display_message(e) if display_message: - raise AirbyteTracedException.from_exception(e, message=display_message) from e + raise AirbyteTracedException.from_exception( + e, message=display_message + ) from e raise e finally: timer.finish_event() @@ -153,7 +197,11 @@ def read( logger.info(timer.report()) if self.continue_sync_on_stream_failure and len(stream_name_to_exception) > 0: - raise AirbyteTracedException(message=self._generate_failed_streams_error_message(stream_name_to_exception)) + raise AirbyteTracedException( + message=self._generate_failed_streams_error_message( + stream_name_to_exception + ) + ) logger.info(f"Finished syncing {self.name}") @property @@ -173,7 +221,9 @@ def _read_stream( internal_config: InternalConfig, ) -> Iterator[AirbyteMessage]: if internal_config.page_size and isinstance(stream_instance, HttpStream): - logger.info(f"Setting page size for {stream_instance.name} to {internal_config.page_size}") + logger.info( + f"Setting page size for {stream_instance.name} to {internal_config.page_size}" + ) stream_instance.page_size = internal_config.page_size logger.debug( f"Syncing configured stream: {configured_stream.stream.name}", @@ -185,7 +235,10 @@ def _read_stream( ) stream_instance.log_stream_sync_configuration() - use_incremental = configured_stream.sync_mode == SyncMode.incremental and stream_instance.supports_incremental + use_incremental = ( + configured_stream.sync_mode == SyncMode.incremental + and stream_instance.supports_incremental + ) if use_incremental: record_iterator = self._read_incremental( logger, @@ -195,7 +248,9 @@ def _read_stream( internal_config, ) else: - record_iterator = self._read_full_refresh(logger, stream_instance, configured_stream, internal_config) + record_iterator = self._read_full_refresh( + logger, stream_instance, configured_stream, internal_config + ) record_counter = 0 stream_name = configured_stream.stream.name @@ -206,7 +261,9 @@ def _read_stream( if record_counter == 1: logger.info(f"Marking stream {stream_name} as RUNNING") # If we just read the first record of the stream, emit the transition to the RUNNING state - yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.RUNNING) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.RUNNING + ) yield from self._emit_queued_messages() yield record @@ -230,7 +287,9 @@ def _read_incremental( :return: """ stream_name = configured_stream.stream.name - stream_state = state_manager.get_stream_state(stream_name, stream_instance.namespace) + stream_state = state_manager.get_stream_state( + stream_name, stream_instance.namespace + ) if stream_state and "state" in dir(stream_instance): stream_instance.state = stream_state # type: ignore # we check that state in the dir(stream_instance) @@ -260,7 +319,9 @@ def _read_full_refresh( internal_config: InternalConfig, ) -> Iterator[AirbyteMessage]: total_records_counter = 0 - for record_data_or_message in stream_instance.read_full_refresh(configured_stream.cursor_field, logger, self._slice_logger): + for record_data_or_message in stream_instance.read_full_refresh( + configured_stream.cursor_field, logger, self._slice_logger + ): message = self._get_message(record_data_or_message, stream_instance) yield message if message.type == MessageType.RECORD: @@ -268,14 +329,21 @@ def _read_full_refresh( if internal_config.is_limit_reached(total_records_counter): return - def _get_message(self, record_data_or_message: Union[StreamData, AirbyteMessage], stream: Stream) -> AirbyteMessage: + def _get_message( + self, record_data_or_message: Union[StreamData, AirbyteMessage], stream: Stream + ) -> AirbyteMessage: """ Converts the input to an AirbyteMessage if it is a StreamData. Returns the input as is if it is already an AirbyteMessage """ if isinstance(record_data_or_message, AirbyteMessage): return record_data_or_message else: - return stream_data_to_airbyte_message(stream.name, record_data_or_message, stream.transformer, stream.get_json_schema()) + return stream_data_to_airbyte_message( + stream.name, + record_data_or_message, + stream.transformer, + stream.get_json_schema(), + ) @property def message_repository(self) -> Union[None, MessageRepository]: @@ -293,6 +361,13 @@ def continue_sync_on_stream_failure(self) -> bool: return False @staticmethod - def _generate_failed_streams_error_message(stream_failures: Mapping[str, AirbyteTracedException]) -> str: - failures = ", ".join([f"{stream}: {exception.__repr__()}" for stream, exception in stream_failures.items()]) + def _generate_failed_streams_error_message( + stream_failures: Mapping[str, AirbyteTracedException] + ) -> str: + failures = ", ".join( + [ + f"{stream}: {exception.__repr__()}" + for stream, exception in stream_failures.items() + ] + ) return f"During the sync, the following streams did not sync successfully: {failures}" diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py index 256e92a61a18..a0d93a02185d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/abstract_source_async.py @@ -4,7 +4,18 @@ import logging from abc import ABC, abstractmethod -from typing import Any, Iterable, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union +from typing import ( + Any, + AsyncGenerator, + Iterable, + Iterator, + List, + Mapping, + MutableMapping, + Optional, + Tuple, + Union, +) from airbyte_cdk.models import ( AirbyteConnectionStatus, @@ -24,7 +35,9 @@ from airbyte_cdk.sources.streams.http.http import HttpStream from airbyte_cdk.sources.utils.record_helper import stream_data_to_airbyte_message from airbyte_cdk.sources.utils.schema_helpers import InternalConfig -from airbyte_cdk.utils.stream_status_utils import as_airbyte_message as stream_status_as_airbyte_message +from airbyte_cdk.utils.stream_status_utils import ( + as_airbyte_message as stream_status_as_airbyte_message, +) class AsyncAbstractSource(AbstractSource, ABC): @@ -34,7 +47,9 @@ class AsyncAbstractSource(AbstractSource, ABC): """ @abstractmethod - async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + async def check_connection( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> Tuple[bool, Optional[Any]]: """ :param logger: source logger :param config: The user-provided configuration as specified by the source's spec. @@ -46,7 +61,9 @@ async def check_connection(self, logger: logging.Logger, config: Mapping[str, An The error object will be cast to string to display the problem to the user. """ - async def check(self, logger: logging.Logger, config: Mapping[str, Any]) -> AirbyteConnectionStatus: + async def check( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> AirbyteConnectionStatus: """Implements the Check Connection operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/#check. """ @@ -68,7 +85,9 @@ async def read( logger: logging.Logger, config: Mapping[str, Any], catalog: ConfiguredAirbyteCatalog, - state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + state: Optional[ + Union[List[AirbyteStateMessage], MutableMapping[str, Any]] + ] = None, ) -> Iterator[AirbyteMessage]: """ Implements the Read operation from the Airbyte Specification. See https://docs.airbyte.com/understanding-airbyte/airbyte-protocol/. @@ -84,9 +103,11 @@ async def read_stream( configured_stream: ConfiguredAirbyteStream, state_manager: ConnectorStateManager, internal_config: InternalConfig, - ) -> Iterator[AirbyteMessage]: + ) -> AsyncGenerator[AirbyteMessage, None]: if internal_config.page_size and isinstance(stream_instance, HttpStream): - logger.info(f"Setting page size for {stream_instance.name} to {internal_config.page_size}") + logger.info( + f"Setting page size for {stream_instance.name} to {internal_config.page_size}" + ) stream_instance.page_size = internal_config.page_size logger.debug( f"Syncing configured stream: {configured_stream.stream.name}", @@ -98,7 +119,10 @@ async def read_stream( ) stream_instance.log_stream_sync_configuration() - use_incremental = configured_stream.sync_mode == SyncMode.incremental and stream_instance.supports_incremental + use_incremental = ( + configured_stream.sync_mode == SyncMode.incremental + and stream_instance.supports_incremental + ) if use_incremental: record_iterator = self._read_incremental( logger, @@ -108,7 +132,9 @@ async def read_stream( internal_config, ) else: - record_iterator = self._read_full_refresh(logger, stream_instance, configured_stream, internal_config) + record_iterator = self._read_full_refresh( + logger, stream_instance, configured_stream, internal_config + ) record_counter = 0 stream_name = configured_stream.stream.name @@ -119,7 +145,9 @@ async def read_stream( if record_counter == 1: logger.info(f"Marking stream {stream_name} as RUNNING") # If we just read the first record of the stream, emit the transition to the RUNNING state - yield stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.RUNNING) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.RUNNING + ) for message in self._emit_queued_messages(): yield message yield record @@ -133,7 +161,7 @@ async def _read_incremental( configured_stream: ConfiguredAirbyteStream, state_manager: ConnectorStateManager, internal_config: InternalConfig, - ) -> Iterator[AirbyteMessage]: + ) -> AsyncGenerator[AirbyteMessage, None]: """Read stream using incremental algorithm :param logger: @@ -144,7 +172,9 @@ async def _read_incremental( :return: """ stream_name = configured_stream.stream.name - stream_state = state_manager.get_stream_state(stream_name, stream_instance.namespace) + stream_state = state_manager.get_stream_state( + stream_name, stream_instance.namespace + ) if stream_state and "state" in dir(stream_instance): stream_instance.state = stream_state # type: ignore # we check that state in the dir(stream_instance) @@ -172,9 +202,11 @@ async def _read_full_refresh( stream_instance: AsyncStream, configured_stream: ConfiguredAirbyteStream, internal_config: InternalConfig, - ) -> Iterator[AirbyteMessage]: + ) -> AsyncGenerator[AirbyteMessage, None]: total_records_counter = 0 - async for record_data_or_message in stream_instance.read_full_refresh(configured_stream.cursor_field, logger, self._slice_logger): + async for record_data_or_message in stream_instance.read_full_refresh( + configured_stream.cursor_field, logger, self._slice_logger + ): message = self._get_message(record_data_or_message, stream_instance) yield message if message.type == MessageType.RECORD: @@ -182,11 +214,20 @@ async def _read_full_refresh( if internal_config.is_limit_reached(total_records_counter): return - def _get_message(self, record_data_or_message: Union[StreamData, AirbyteMessage], stream: AsyncStream) -> AirbyteMessage: + def _get_message( + self, + record_data_or_message: Union[StreamData, AirbyteMessage], + stream: AsyncStream, + ) -> AirbyteMessage: """ Converts the input to an AirbyteMessage if it is a StreamData. Returns the input as is if it is already an AirbyteMessage """ if isinstance(record_data_or_message, AirbyteMessage): return record_data_or_message else: - return stream_data_to_airbyte_message(stream.name, record_data_or_message, stream.transformer, stream.get_json_schema()) + return stream_data_to_airbyte_message( + stream.name, + record_data_or_message, + stream.transformer, + stream.get_json_schema(), + ) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py index a8fd8bc02217..83618b290e5d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py @@ -6,9 +6,26 @@ import logging from abc import ABC from queue import Queue -from typing import Any, Dict, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union +from typing import ( + Any, + Dict, + Iterator, + List, + Mapping, + MutableMapping, + Optional, + Tuple, + Union, +) -from airbyte_cdk.models import AirbyteMessage, AirbyteStateMessage, AirbyteStreamStatus, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, ConnectorSpecification +from airbyte_cdk.models import ( + AirbyteMessage, + AirbyteStateMessage, + AirbyteStreamStatus, + ConfiguredAirbyteCatalog, + ConfiguredAirbyteStream, + ConnectorSpecification, +) from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.abstract_source import AbstractSource from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource @@ -17,7 +34,9 @@ from airbyte_cdk.sources.utils.schema_helpers import InternalConfig, split_config from airbyte_cdk.utils.event_timing import create_timer from airbyte_cdk.utils.traced_exception import AirbyteTracedException -from airbyte_cdk.utils.stream_status_utils import as_airbyte_message as stream_status_as_airbyte_message +from airbyte_cdk.utils.stream_status_utils import ( + as_airbyte_message as stream_status_as_airbyte_message, +) DEFAULT_QUEUE_SIZE = 10000 DEFAULT_SESSION_LIMIT = 10000 @@ -34,12 +53,16 @@ def __init__(self, async_source: AsyncAbstractSource): self.queue = Queue(DEFAULT_QUEUE_SIZE) self.session_limit = DEFAULT_SESSION_LIMIT - def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + def check_connection( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> Tuple[bool, Optional[Any]]: """ Run the async_source's `check_connection` method on the event loop. """ loop = asyncio.get_event_loop() - return loop.run_until_complete(self.async_source.check_connection(logger, config)) + return loop.run_until_complete( + self.async_source.check_connection(logger, config) + ) def streams(self, config: Mapping[str, Any]) -> List[AsyncStream]: """ @@ -59,7 +82,9 @@ def read( logger: logging.Logger, config: Mapping[str, Any], catalog: ConfiguredAirbyteCatalog, - state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + state: Optional[ + Union[List[AirbyteStateMessage], MutableMapping[str, Any]] + ] = None, ) -> Iterator[AirbyteMessage]: """ Run the async_source's `read_streams` method and yield its results. @@ -67,25 +92,54 @@ def read( """ logger.info(f"Starting syncing {self.name}") config, internal_config = split_config(config) - stream_instances = {s.name: s for s in self.streams(config)} - state_manager = ConnectorStateManager(stream_instance_map=stream_instances, state=state) + stream_instances: Mapping[str, AsyncStream] = { + s.name: s for s in self.streams(config) + } + state_manager = ConnectorStateManager( + stream_instance_map=stream_instances, state=state + ) self._stream_to_instance_map = stream_instances self._assert_streams(catalog, stream_instances) n_records = 0 with create_timer(self.name) as timer: - for record in self._do_read(catalog, stream_instances, timer, logger, state_manager, internal_config): + for record in self._do_read( + catalog, stream_instances, timer, logger, state_manager, internal_config + ): n_records += 1 yield record print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") logger.info(f"Finished syncing {self.name}") - def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): - streams_in_progress_sentinels = {s.stream.name: Sentinel(s.stream.name) for s in catalog.streams if s.stream.name in stream_instances} + def _do_read( + self, + catalog: ConfiguredAirbyteCatalog, + stream_instances: Dict[str, AsyncStream], + timer: Any, + logger: logging.Logger, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ): + streams_in_progress_sentinels = { + s.stream.name: Sentinel(s.stream.name) + for s in catalog.streams + if s.stream.name in stream_instances + } if not streams_in_progress_sentinels: return - self.reader = SourceReader(logger, self.queue, streams_in_progress_sentinels, self._read_streams, catalog, stream_instances, timer, logger, state_manager, internal_config) + self.reader = SourceReader( + logger, + self.queue, + streams_in_progress_sentinels, + self._read_streams, + catalog, + stream_instances, + timer, + logger, + state_manager, + internal_config, + ) for record in self.reader: yield record @@ -94,7 +148,11 @@ def _do_read(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str raise record yield record - def _assert_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream]): + def _assert_streams( + self, + catalog: ConfiguredAirbyteCatalog, + stream_instances: Dict[str, AsyncStream], + ): for configured_stream in catalog.streams: stream_instance = stream_instances.get(configured_stream.stream.name) if not stream_instance: @@ -105,23 +163,48 @@ def _assert_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: D f"Refresh the schema in replication settings and remove this stream from future sync attempts." ) - async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + async def _read_streams( + self, + catalog: ConfiguredAirbyteCatalog, + stream_instances: Dict[str, AsyncStream], + timer: Any, + logger: logging.Logger, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ): pending_tasks = set() n_started, n_streams = 0, len(catalog.streams) streams_iterator = iter(catalog.streams) exceptions = False while (pending_tasks or n_started < n_streams) and not exceptions: - while len(pending_tasks) < self.session_limit and (configured_stream := next(streams_iterator, None)): + while len(pending_tasks) < self.session_limit and ( + configured_stream := next(streams_iterator, None) + ): if configured_stream is None: break stream_instance = stream_instances.get(configured_stream.stream.name) stream = stream_instances.get(configured_stream.stream.name) - self.reader.sessions[configured_stream.stream.name] = await stream.ensure_session() - pending_tasks.add(asyncio.create_task(self._do_async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config))) + self.reader.sessions[ + configured_stream.stream.name + ] = await stream.ensure_session() + pending_tasks.add( + asyncio.create_task( + self._do_async_read_stream( + configured_stream, + stream_instance, + timer, + logger, + state_manager, + internal_config, + ) + ) + ) n_started += 1 - done, pending_tasks = await asyncio.wait(pending_tasks, return_when=asyncio.FIRST_COMPLETED) + done, pending_tasks = await asyncio.wait( + pending_tasks, return_when=asyncio.FIRST_COMPLETED + ) for task in done: if exc := task.exception(): @@ -130,21 +213,52 @@ async def _read_streams(self, catalog: ConfiguredAirbyteCatalog, stream_instance self.queue.put(exc) exceptions = True - async def _do_async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + async def _do_async_read_stream( + self, + configured_stream: ConfiguredAirbyteStream, + stream_instance: AsyncStream, + timer: Any, + logger: logging.Logger, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ): try: - await self._async_read_stream(configured_stream, stream_instance, timer, logger, state_manager, internal_config) + await self._async_read_stream( + configured_stream, + stream_instance, + timer, + logger, + state_manager, + internal_config, + ) finally: self.queue.put(Sentinel(configured_stream.stream.name)) - async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, stream_instance: AsyncStream, timer: Any, logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig): + async def _async_read_stream( + self, + configured_stream: ConfiguredAirbyteStream, + stream_instance: AsyncStream, + timer: Any, + logger: logging.Logger, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ): try: timer.start_event(f"Syncing stream {configured_stream.stream.name}") - stream_is_available, reason = await stream_instance.check_availability(logger, self) + stream_is_available, reason = await stream_instance.check_availability( + logger, self.async_source + ) if not stream_is_available: - logger.warning(f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}") + logger.warning( + f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}" + ) return logger.info(f"Marking stream {configured_stream.stream.name} as STARTED") - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.STARTED)) + self.queue.put( + stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.STARTED + ) + ) async for record in self.async_source.read_stream( logger=logger, stream_instance=stream_instance, @@ -154,16 +268,30 @@ async def _async_read_stream(self, configured_stream: ConfiguredAirbyteStream, s ): self.queue.put(record) logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.COMPLETE)) + self.queue.put( + stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.COMPLETE + ) + ) except AirbyteTracedException as e: - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + self.queue.put( + stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.INCOMPLETE + ) + ) raise e except Exception as e: for message in self._emit_queued_messages(): self.queue.put(message) - logger.exception(f"Encountered an exception while reading stream {configured_stream.stream.name}") + logger.exception( + f"Encountered an exception while reading stream {configured_stream.stream.name}" + ) logger.info(f"Marking stream {configured_stream.stream.name} as STOPPED") - self.queue.put(stream_status_as_airbyte_message(configured_stream.stream, AirbyteStreamStatus.INCOMPLETE)) + self.queue.put( + stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.INCOMPLETE + ) + ) display_message = await stream_instance.get_error_display_message(e) if display_message: raise AirbyteTracedException.from_exception(e, message=display_message) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/async_call_rate.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/async_call_rate.py index d690c153c152..bb0669120f8c 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/async_call_rate.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/async_call_rate.py @@ -3,7 +3,7 @@ # import logging -from typing import Any +from typing import Any, Optional import aiohttp import aiohttp_client_cache @@ -20,23 +20,27 @@ class AsyncLimiterMixin(MIXIN_BASE): def __init__( self, - api_budget: AbstractAPIBudget, + api_budget: Optional[AbstractAPIBudget], **kwargs: Any, ): self._api_budget = api_budget super().__init__(**kwargs) # type: ignore # Base Session doesn't take any kwargs - async def send(self, request: aiohttp.ClientRequest, **kwargs: Any) -> aiohttp.ClientResponse: + async def send( + self, request: aiohttp.ClientRequest, **kwargs: Any + ) -> aiohttp.ClientResponse: """Send a request with rate-limiting.""" - self._api_budget.acquire_call(request) - response = await super().send(request, **kwargs) - self._api_budget.update_from_response(request, response) - return response + assert ( + self._api_budget is None + ), "API budgets are not supported in the async CDK yet." + return await super().send(request, **kwargs) # type: ignore # MIXIN_BASE should be used with aiohttp.ClientSession class AsyncLimiterSession(AsyncLimiterMixin, aiohttp.ClientSession): """Session that adds rate-limiting behavior to requests.""" -class AsyncCachedLimiterSession(aiohttp_client_cache.CachedSession, AsyncLimiterMixin, aiohttp.ClientSession): +class AsyncCachedLimiterSession( + aiohttp_client_cache.CachedSession, AsyncLimiterMixin, aiohttp.ClientSession +): """Session class with caching and rate-limiting behavior.""" diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/availability_strategy_async.py index 3fcc588149ae..23377887b95f 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/availability_strategy_async.py @@ -3,14 +3,11 @@ # import logging -import typing from abc import ABC, abstractmethod from typing import Optional, Tuple -from airbyte_cdk.sources.streams import Stream - -if typing.TYPE_CHECKING: - from airbyte_cdk.sources import Source +from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource +from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream class AsyncAvailabilityStrategy(ABC): @@ -19,7 +16,12 @@ class AsyncAvailabilityStrategy(ABC): """ @abstractmethod - async def check_availability(self, stream: Stream, logger: logging.Logger, source: Optional["Source"]) -> Tuple[bool, Optional[str]]: + async def check_availability( + self, + stream: AsyncStream, + logger: logging.Logger, + source: Optional["AsyncAbstractSource"], + ) -> Tuple[bool, Optional[str]]: """ Checks stream availability. diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py index 9d0ad38483f0..791135410763 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py @@ -5,7 +5,16 @@ import logging from abc import ABC, abstractmethod -from typing import TYPE_CHECKING, Any, Iterable, List, Mapping, MutableMapping, Optional, Tuple +from typing import ( + TYPE_CHECKING, + Any, + AsyncGenerator, + List, + Mapping, + MutableMapping, + Optional, + Tuple, +) from airbyte_cdk.models import SyncMode from airbyte_cdk.models import Type as MessageType @@ -14,7 +23,10 @@ from airbyte_cdk.sources.utils.slice_logger import SliceLogger if TYPE_CHECKING: - from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy + from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource + from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import ( + AsyncHttpAvailabilityStrategy, + ) class AsyncStream(Stream, ABC): @@ -27,9 +39,14 @@ async def read_full_refresh( cursor_field: Optional[List[str]], logger: logging.Logger, slice_logger: SliceLogger, - ) -> Iterable[StreamData]: - async for _slice in self.stream_slices(sync_mode=SyncMode.full_refresh, cursor_field=cursor_field): - logger.debug(f"Processing stream slices for {self.name} (sync_mode: full_refresh)", extra={"stream_slice": _slice}) + ) -> AsyncGenerator[StreamData, None]: + async for _slice in self.stream_slices( + sync_mode=SyncMode.full_refresh, cursor_field=cursor_field + ): + logger.debug( + f"Processing stream slices for {self.name} (sync_mode: full_refresh)", + extra={"stream_slice": _slice}, + ) if slice_logger.should_log_slice_message(logger): yield slice_logger.create_slice_log_message(_slice) async for record in self.read_records( @@ -48,13 +65,16 @@ async def read_incremental( # type: ignore # ignoring typing for ConnectorStat state_manager, per_stream_state_enabled: bool, internal_config: InternalConfig, - ) -> Iterable[StreamData]: + ) -> AsyncGenerator[StreamData, None]: slices = self.stream_slices( cursor_field=cursor_field, sync_mode=SyncMode.incremental, stream_state=stream_state, ) - logger.debug(f"Processing stream slices for {self.name} (sync_mode: incremental)", extra={"stream_slices": slices}) + logger.debug( + f"Processing stream slices for {self.name} (sync_mode: incremental)", + extra={"stream_slices": slices}, + ) has_slices = False record_counter = 0 @@ -71,23 +91,37 @@ async def read_incremental( # type: ignore # ignoring typing for ConnectorStat async for record_data_or_message in records: yield record_data_or_message if isinstance(record_data_or_message, Mapping) or ( - hasattr(record_data_or_message, "type") and record_data_or_message.type == MessageType.RECORD + hasattr(record_data_or_message, "type") + and record_data_or_message.type == MessageType.RECORD ): - record_data = record_data_or_message if isinstance(record_data_or_message, Mapping) else record_data_or_message.record + record_data = ( + record_data_or_message + if isinstance(record_data_or_message, Mapping) + else record_data_or_message.record + ) stream_state = self.get_updated_state(stream_state, record_data) checkpoint_interval = self.state_checkpoint_interval record_counter += 1 - if checkpoint_interval and record_counter % checkpoint_interval == 0: - yield self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) + if ( + checkpoint_interval + and record_counter % checkpoint_interval == 0 + ): + yield self._checkpoint_state( + stream_state, state_manager, per_stream_state_enabled + ) if internal_config.is_limit_reached(record_counter): break - yield self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) + yield self._checkpoint_state( + stream_state, state_manager, per_stream_state_enabled + ) if not has_slices: # Safety net to ensure we always emit at least one state message even if there are no slices - checkpoint = self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) + checkpoint = self._checkpoint_state( + stream_state, state_manager, per_stream_state_enabled + ) yield checkpoint @abstractmethod @@ -97,14 +131,19 @@ async def read_records( cursor_field: Optional[List[str]] = None, stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, - ) -> Iterable[StreamData]: + ) -> AsyncGenerator[StreamData, None]: """ This method should be overridden by subclasses to read records based on the inputs """ + ... async def stream_slices( - self, *, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None - ) -> Iterable[Optional[Mapping[str, Any]]]: + self, + *, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> AsyncGenerator[Optional[Mapping[str, Any]], None]: """ Override to define the slices for this stream. See the stream slicing section of the docs for more information. @@ -128,7 +167,9 @@ def availability_strategy(self) -> Optional["AsyncHttpAvailabilityStrategy"]: """ return None - async def check_availability(self, logger: logging.Logger, source: Optional["Source"] = None) -> Tuple[bool, Optional[str]]: + async def check_availability( + self, logger: logging.Logger, source: Optional["AsyncAbstractSource"] = None + ) -> Tuple[bool, Optional[str]]: """ Checks whether this stream is available. @@ -140,10 +181,14 @@ async def check_availability(self, logger: logging.Logger, source: Optional["Sou resolve the unavailability, if possible. """ if self.availability_strategy: - return await self.availability_strategy.check_availability(self, logger, source) + return await self.availability_strategy.check_availability( + self, logger, source + ) return True, None - async def get_error_display_message(self, exception: BaseException) -> Optional[str]: + async def get_error_display_message( + self, exception: BaseException + ) -> Optional[str]: """ Retrieves the user-friendly display message that corresponds to an exception. This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py index d024c1fccbb8..468925bcd30a 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py @@ -3,23 +3,28 @@ # import logging -import typing from typing import Dict, Optional, Tuple import requests from aiohttp import ClientResponseError from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream -from airbyte_cdk.sources.async_cdk.streams.utils.stream_helper_async import get_first_record_for_slice, get_first_stream_slice -from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy - - -if typing.TYPE_CHECKING: - from airbyte_cdk.sources import Source +from airbyte_cdk.sources.async_cdk.streams.utils.stream_helper_async import ( + get_first_record_for_slice, + get_first_stream_slice, +) +from airbyte_cdk.sources.streams.http.availability_strategy import ( + HttpAvailabilityStrategy, +) class AsyncHttpAvailabilityStrategy(HttpAvailabilityStrategy): - async def check_availability(self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"]) -> Tuple[bool, Optional[str]]: + async def check_availability( + self, + stream: AsyncStream, + logger: logging.Logger, + source: Optional["AsyncAbstractSource"], + ) -> Tuple[bool, Optional[str]]: """ Check stream availability by attempting to read the first record of the stream. @@ -43,7 +48,9 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, reason = f"Cannot attempt to connect to stream {stream.name} - no stream slices were found, likely because the parent stream is empty." return False, reason except ClientResponseError as error: - is_available, reason = await self._handle_http_error(stream, logger, source, error) + is_available, reason = await self._handle_http_error( + stream, logger, source, error + ) if not is_available: reason = f"Unable to get slices for {stream.name} stream, because of error in parent stream. {reason}" return is_available, reason @@ -52,10 +59,14 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, async for _ in get_first_record_for_slice(stream, stream_slice): return True, None except StopAsyncIteration: - logger.info(f"Successfully connected to stream {stream.name}, but got 0 records.") + logger.info( + f"Successfully connected to stream {stream.name}, but got 0 records." + ) return True, None except ClientResponseError as error: - is_available, reason = await self._handle_http_error(stream, logger, source, error) + is_available, reason = await self._handle_http_error( + stream, logger, source, error + ) if not is_available: reason = f"Unable to read {stream.name} stream. {reason}" return is_available, reason @@ -63,7 +74,11 @@ async def check_availability(self, stream: AsyncStream, logger: logging.Logger, return True, None async def _handle_http_error( - self, stream: AsyncStream, logger: logging.Logger, source: Optional["AsyncAbstractSource"], error: ClientResponseError + self, + stream: AsyncStream, + logger: logging.Logger, + source: Optional["AsyncAbstractSource"], + error: ClientResponseError, ) -> Tuple[bool, Optional[str]]: """ Override this method to define error handling for various `HTTPError`s @@ -82,7 +97,9 @@ async def _handle_http_error( resolve the unavailability, if possible. """ status_code = error.status - known_status_codes = self.reasons_for_unavailable_status_codes(stream, logger, source, error) + known_status_codes = self.reasons_for_unavailable_status_codes( + stream, logger, source, error + ) known_reason = known_status_codes.get(status_code) if not known_reason: # If the HTTPError is not in the dictionary of errors we know how to handle, don't except @@ -93,7 +110,11 @@ async def _handle_http_error( return False, reason def reasons_for_unavailable_status_codes( - self, stream: AsyncStream, logger: logging.Logger, source: Optional["Source"], error: ClientResponseError + self, + stream: AsyncStream, + logger: logging.Logger, + source: Optional["AsyncAbstractSource"], + error: ClientResponseError, ) -> Dict[int, str]: """ Returns a dictionary of HTTP status codes that indicate stream diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py index 1204ce4cc7c6..1c2d10915160 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py @@ -7,16 +7,36 @@ import os from abc import ABC, abstractmethod from pathlib import Path -from typing import Any, Callable, Iterable, List, Mapping, Optional, Tuple, TypeVar, Union +from typing import ( + Any, + AsyncGenerator, + Callable, + Dict, + List, + Mapping, + Optional, + Tuple, + TypeVar, + Union, +) from yarl import URL import aiohttp import aiohttp_client_cache from airbyte_cdk.models import SyncMode -from airbyte_cdk.sources.async_cdk.streams.async_call_rate import AsyncCachedLimiterSession, AsyncLimiterSession +from airbyte_cdk.sources.async_cdk.streams.async_call_rate import ( + AsyncCachedLimiterSession, + AsyncLimiterSession, +) from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream -from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy -from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException +from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import ( + AsyncHttpAvailabilityStrategy, +) +from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import ( + DefaultBackoffException, + RequestBodyException, + UserDefinedBackoffException, +) from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.auth import NoAuth @@ -29,7 +49,27 @@ # list of all possible HTTP methods which can be used for sending of request bodies BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") -T = TypeVar("T") +T1 = TypeVar("T1") +T2 = TypeVar("T2") +T3 = TypeVar("T3") +T4 = TypeVar("T4") +RecordsGeneratorFunction = Callable[ + [ + aiohttp.ClientRequest, + aiohttp.ClientResponse, + Mapping[str, Any], + Optional[Mapping[str, Any]], + ], + AsyncGenerator[StreamData, None], +] + + +""" + records_generator_fn: Callable[ + , + Iterable[StreamData], + ], +""" class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): @@ -39,10 +79,12 @@ class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): Basic building block for users building an Airbyte source for an async HTTP API. """ - def __init__(self, authenticator: Optional[Union[HttpAuthenticator, NoAuth]] = NoAuth()): + def __init__( + self, authenticator: Optional[Union[HttpAuthenticator, NoAuth]] = NoAuth() + ): # TODO: wire in rate limiting via ApiBudget self._api_budget = None - self._session: aiohttp.ClientSession = None + self._session: Optional[aiohttp.ClientSession] = None # TODO: HttpStream handles other authentication codepaths, which may need to be added later self._authenticator = authenticator @@ -69,7 +111,7 @@ def request_session(self) -> aiohttp.ClientSession: kwargs = {} if self._authenticator: - kwargs['headers'] = self._authenticator.get_auth_header() + kwargs["headers"] = self._authenticator.get_auth_header() if self.use_cache: cache_dir = os.getenv(ENV_REQUEST_CACHE_PATH) @@ -79,10 +121,25 @@ def request_session(self) -> aiohttp.ClientSession: sqlite_path = str(Path(cache_dir) / self.cache_filename) else: sqlite_path = "file::memory:?cache=shared" - cache = aiohttp_client_cache.SQLiteBackend(cache_name=sqlite_path, allowed_methods=("get", "post", "put", "patch", "options", "delete", "list")) - return AsyncCachedLimiterSession(cache=cache, connector=connector, api_budget=self._api_budget) + cache = aiohttp_client_cache.SQLiteBackend( + cache_name=sqlite_path, + allowed_methods=( + "get", + "post", + "put", + "patch", + "options", + "delete", + "list", + ), + ) + return AsyncCachedLimiterSession( + cache=cache, connector=connector, api_budget=self._api_budget + ) else: - return AsyncLimiterSession(connector=connector, api_budget=self._api_budget, **kwargs) + return AsyncLimiterSession( + connector=connector, api_budget=self._api_budget, **kwargs + ) async def clear_cache(self) -> None: """ @@ -92,7 +149,9 @@ async def clear_cache(self) -> None: await self._session.cache.clear() @abstractmethod - async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + async def next_page_token( + self, response: aiohttp.ClientResponse + ) -> Optional[Mapping[str, Any]]: """ Override this method to define a pagination strategy. @@ -109,7 +168,7 @@ async def parse_response( stream_state: Mapping[str, Any], stream_slice: Optional[Mapping[str, Any]] = None, next_page_token: Optional[Mapping[str, Any]] = None, - ) -> List[Mapping[str, Any]]: + ) -> AsyncGenerator[Mapping[str, Any], None]: """ Parses the raw response object into a list of records. By default, this returns an iterable containing the input. Override to parse differently. @@ -119,6 +178,7 @@ async def parse_response( :param next_page_token: :return: An iterable containing the parsed response """ + ... # TODO move all the retry logic to a functor/decorator which is input as an init parameter def should_retry(self, response: aiohttp.ClientResponse) -> bool: @@ -157,7 +217,7 @@ def error_message(self, response: aiohttp.ClientResponse) -> str: def _create_prepared_request( self, path: str, - headers: Optional[Mapping[str, str]] = None, + headers: Optional[Dict[str, str]] = None, params: Optional[Mapping[str, str]] = None, json: Optional[Mapping[str, Any]] = None, data: Optional[Union[str, Mapping[str, Any]]] = None, @@ -167,7 +227,7 @@ def _create_prepared_request( def _create_aiohttp_client_request( self, path: str, - headers: Optional[Mapping[str, str]] = None, + headers: Optional[Dict[str, str]] = None, params: Optional[Mapping[str, str]] = None, json_data: Optional[Mapping[str, Any]] = None, data: Optional[Union[str, Mapping[str, Any]]] = None, @@ -186,14 +246,18 @@ def _create_aiohttp_client_request( ) elif json_data: headers = headers or {} - headers.update({'Content-Type': 'application/json'}) + headers.update({"Content-Type": "application/json"}) data = json.dumps(json_data) - client_request = aiohttp.ClientRequest(self.http_method, url, headers=headers, params=query_params, data=data) + client_request = aiohttp.ClientRequest( + self.http_method, url, headers=headers, params=query_params, data=data + ) return client_request - async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any]) -> aiohttp.ClientResponse: + async def _send( + self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any] + ) -> aiohttp.ClientResponse: """ Wraps sending the request in rate limit and error handlers. Please note that error handling for HTTP status codes will be ignored if raise_on_http_errors is set to False @@ -213,11 +277,21 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st Unexpected persistent exceptions are not handled and will cause the sync to fail. """ self.logger.debug( - "Making outbound API request", extra={"headers": request.headers, "url": request.url, "request_body": request.body} + "Making outbound API request", + extra={ + "headers": request.headers, + "url": request.url, + "request_body": request.body, + }, ) + if self._session is None: + raise AssertionError( + "The session was not set before attempting to make a request. This is unexpected. Please contact Support." + ) response = await self._session.request( - request.method, request.url, + request.method, + request.url, headers=request.headers, auth=request.auth, **request_kwargs, @@ -227,17 +301,26 @@ async def _send(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[st # Do it only in debug mode if self.logger.isEnabledFor(logging.DEBUG): self.logger.debug( - "Receiving response", extra={"headers": response.headers, "status": response.status, "body": response.text} + "Receiving response", + extra={ + "headers": response.headers, + "status": response.status, + "body": response.text, + }, ) if self.should_retry(response): custom_backoff_time = self.backoff_time(response) error_message = self.error_message(response) if custom_backoff_time: raise UserDefinedBackoffException( - backoff=custom_backoff_time, response=response, error_message=error_message + backoff=custom_backoff_time, + response=response, + error_message=error_message, ) else: - raise DefaultBackoffException(response=response, error_message=error_message) + raise DefaultBackoffException( + response=response, error_message=error_message + ) elif self.raise_on_http_errors: # Raise any HTTP exceptions that happened in case there were unexpected ones return await self.handle_response_with_error(response) @@ -248,7 +331,9 @@ async def ensure_session(self) -> aiohttp.ClientSession: self._session = self.request_session() return self._session - async def _send_request(self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any]) -> aiohttp.ClientResponse: + async def _send_request( + self, request: aiohttp.ClientRequest, request_kwargs: Mapping[str, Any] + ) -> aiohttp.ClientResponse: """ Creates backoff wrappers which are responsible for retry logic """ @@ -284,15 +369,18 @@ async def _send_request(self, request: aiohttp.ClientRequest, request_kwargs: Ma if max_tries is not None: max_tries = max(0, max_tries) + 1 - @default_backoff_handler(max_tries=max_tries, max_time=max_time, factor=self.retry_factor) - @user_defined_backoff_handler(max_tries=max_tries, max_time=max_time) - async def send(): - return await self._send(request, request_kwargs) - - return await send() + user_backoff_handler = user_defined_backoff_handler( + max_tries=max_tries, max_time=max_time + )(self._send) + backoff_handler = default_backoff_handler( + max_tries=max_tries, max_time=max_time, factor=self.retry_factor + ) + return await backoff_handler(user_backoff_handler)(request, request_kwargs) @classmethod - async def parse_response_error_message(cls, exception: aiohttp.ClientResponseError) -> Optional[str]: + async def parse_response_error_message( + cls, exception: aiohttp.ClientResponseError + ) -> Optional[str]: """ Parses the raw response object from a failed request into a user-friendly error message. By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. @@ -300,6 +388,7 @@ async def parse_response_error_message(cls, exception: aiohttp.ClientResponseErr :param response: :return: A user-friendly message that indicates the cause of the error """ + # default logic to grab error from common fields def _try_get_error(value: Optional[JsonType]) -> Optional[str]: if isinstance(value, str): @@ -322,13 +411,17 @@ def _try_get_error(value: Optional[JsonType]) -> Optional[str]: try: if hasattr(exception, "_response_error"): - return _try_get_error(exception._response_error) + return _try_get_error(exception._response_error) # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 else: - raise NotImplementedError("`_response_error` is expected but was not set on the response; `handle_response_with_error` should be used prior to processing the exception") + raise NotImplementedError( + "`_response_error` is expected but was not set on the response; `handle_response_with_error` should be used prior to processing the exception" + ) except json.JSONDecodeError: return None - async def get_error_display_message(self, exception: BaseException) -> Optional[str]: + async def get_error_display_message( + self, exception: BaseException + ) -> Optional[str]: """ Retrieves the user-friendly display message that corresponds to an exception. This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. @@ -339,7 +432,10 @@ async def get_error_display_message(self, exception: BaseException) -> Optional[ :param exception: The exception that was raised :return: A user-friendly message that indicates the cause of the error """ - if isinstance(exception, aiohttp.ClientResponseError) and exception.message is not None: + if ( + isinstance(exception, aiohttp.ClientResponseError) + and exception.message is not None + ): return await self.parse_response_error_message(exception) return None @@ -349,35 +445,48 @@ async def read_records( cursor_field: Optional[List[str]] = None, stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, - ) -> Iterable[StreamData]: - async def _records_generator_fn(req, res, state, _slice): - async for record in self.parse_response(res, stream_slice=_slice, stream_state=state): + ) -> AsyncGenerator[StreamData, None]: + async def _records_generator_fn( + req, res, state, _slice + ) -> AsyncGenerator[StreamData, None]: + async for record in self.parse_response( + res, stream_slice=_slice, stream_state=state + ): yield record - async for record in self._read_pages(_records_generator_fn, stream_slice, stream_state): + async for record in self._read_pages( + _records_generator_fn, stream_slice, stream_state + ): yield record async def _read_pages( self, - records_generator_fn: Callable[ - [aiohttp.ClientRequest, aiohttp.ClientResponse, Mapping[str, Any], Optional[Mapping[str, Any]]], Iterable[StreamData] - ], + records_generator_fn: RecordsGeneratorFunction, stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, - ) -> Iterable[StreamData]: + ) -> AsyncGenerator[StreamData, None]: stream_state = stream_state or {} pagination_complete = False next_page_token = None while not pagination_complete: - async def f(): + + async def f() -> Tuple[ + aiohttp.ClientRequest, + aiohttp.ClientResponse, + Optional[Mapping[str, Any]], + ]: nonlocal next_page_token - request, response = await self._fetch_next_page(stream_slice, stream_state, next_page_token) + request, response = await self._fetch_next_page( + stream_slice, stream_state, next_page_token + ) next_page_token = await self.next_page_token(response) return request, response, next_page_token request, response, next_page_token = await f() - async for record in records_generator_fn(request, response, stream_state, stream_slice): + async for record in records_generator_fn( + request, response, stream_state, stream_slice + ): yield record if not next_page_token: @@ -389,20 +498,49 @@ async def _fetch_next_page( stream_state: Optional[Mapping[str, Any]] = None, next_page_token: Optional[Mapping[str, Any]] = None, ) -> Tuple[aiohttp.ClientRequest, aiohttp.ClientResponse]: - request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + request_headers = self.request_headers( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ) request = self._create_prepared_request( - path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), - headers=dict(request_headers, **self.authenticator.get_auth_header()), - params=self.request_params(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), - json=self.request_body_json(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), - data=self.request_body_data(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + path=self.path( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + headers=dict( + request_headers, + **self.authenticator.get_auth_header() if self.authenticator else {}, + ), + params=self.request_params( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + json=self.request_body_json( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + data=self.request_body_data( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + ) + request_kwargs = self.request_kwargs( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, ) - request_kwargs = self.request_kwargs(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) response = await self._send_request(request, request_kwargs) return request, response - async def handle_response_with_error(self, response: aiohttp.ClientResponse) -> aiohttp.ClientResponse: + async def handle_response_with_error( + self, response: aiohttp.ClientResponse + ) -> aiohttp.ClientResponse: """ If the response has a non-ok status code, raise an exception, otherwise return the response. @@ -423,11 +561,13 @@ async def handle_response_with_error(self, response: aiohttp.ClientResponse) -> response.request_info, response.history, status=response.status, - message=response.reason, + message=response.reason or "", headers=response.headers, ) text = await response.text() - exc._response_error = error_json or text # https://github.com/aio-libs/aiohttp/issues/3248 + exc._response_error = ( # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 + error_json or text + ) self.logger.error(text) raise exc @@ -441,15 +581,23 @@ def __init__(self, parent: AsyncHttpStream, **kwargs: Any): self.parent = parent async def stream_slices( - self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None - ) -> Iterable[Optional[Mapping[str, Any]]]: + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_state: Optional[Mapping[str, Any]] = None, + ) -> AsyncGenerator[Optional[Mapping[str, Any]], None]: await self.parent.ensure_session() # iterate over all parent stream_slices async for stream_slice in self.parent.stream_slices( - sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_state=stream_state + sync_mode=SyncMode.full_refresh, + cursor_field=cursor_field, + stream_state=stream_state, ): parent_records = self.parent.read_records( - sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_slice=stream_slice, stream_state=stream_state + sync_mode=SyncMode.full_refresh, + cursor_field=cursor_field, + stream_slice=stream_slice, + stream_state=stream_state, ) # iterate over all parent records with current stream_slice diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py index 18721aa0420b..c5bd5f3816be 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py @@ -23,32 +23,41 @@ aiohttp.ServerTimeoutError, aiohttp.ServerConnectionError, aiohttp.ServerDisconnectedError, - ) logger = logging.getLogger("airbyte") -SendRequestCallableType = Callable[[aiohttp.ClientRequest, Mapping[str, Any]], Coroutine[Any, Any, aiohttp.ClientResponse]] +SendRequestCallableType = Callable[ + [aiohttp.ClientRequest, Mapping[str, Any]], + Coroutine[Any, Any, aiohttp.ClientResponse], +] TOO_MANY_REQUESTS_CODE = 429 + def default_backoff_handler( - max_tries: Optional[int], factor: float, max_time: Optional[int] = None, **kwargs: Any + max_tries: Optional[int], + factor: float, + max_time: Optional[int] = None, + **kwargs: Any, ) -> Callable[[SendRequestCallableType], SendRequestCallableType]: def log_retry_attempt(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, aiohttp.ClientResponseError) and exc.history: - logger.info(f"Status code: {exc.status}, Response Content: {'FIX ME'}") # TODO + logger.info( + f"Status code: {exc.status}, Response Content: {'FIX ME'}" + ) # TODO logger.info( f"Caught retryable error '{str(exc)}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." ) def should_give_up(exc: Exception) -> bool: - from requests import codes # If a non-rate-limiting related 4XX error makes it this far, it means it was unexpected and probably consistent, so we shouldn't back off if isinstance(exc, aiohttp.ClientResponseError): give_up: bool = ( - exc.history is not None and exc.status != TOO_MANY_REQUESTS_CODE and 400 <= exc.status < 500 + exc.history is not None + and exc.status != TOO_MANY_REQUESTS_CODE + and 400 <= exc.status < 500 ) if give_up: logger.info(f"Giving up for returned HTTP status: {exc.status}") @@ -76,7 +85,9 @@ def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, UserDefinedBackoffException): if exc.history: - logger.info(f"Status code: {exc.status}, Response Content: {'FIX ME'}") # TODO + logger.info( + f"Status code: {exc.status}, Response Content: {'FIX ME'}" # TODO + ) # TODO retry_after = exc.backoff logger.info(f"Retrying. Sleeping for {retry_after} seconds") time.sleep(retry_after + 1) # extra second to cover any fractions of second @@ -84,7 +95,9 @@ def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: def log_give_up(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, aiohttp.ClientResponseError): - logger.error(f"Max retry limit reached. Request: {exc.request_info}, Response: {exc.history}") # TODO: how does history get printed out + logger.error( + f"Max retry limit reached. Request: {exc.request_info}, Response: {exc.history}" + ) # TODO: how does history get printed out else: logger.error("Max retry limit reached for unknown request and response") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py index 293c12ab620a..4fb97d78a919 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py @@ -2,24 +2,34 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # -from typing import Any, Mapping, Optional +from typing import Any, AsyncGenerator, Mapping, Optional from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.streams.core import StreamData -async def get_first_stream_slice(stream: AsyncStream) -> Mapping[str, Any]: +async def get_first_stream_slice(stream: AsyncStream) -> Optional[Mapping[str, Any]]: """ Gets the first stream_slice from a given stream's stream_slices. :param stream: stream :raises StopAsyncIteration: if there is no first slice to return (the stream_slices generator is empty) :return: first stream slice from 'stream_slices' generator (`None` is a valid stream slice) """ - return await anext(stream.stream_slices(cursor_field=stream.cursor_field, sync_mode=SyncMode.full_refresh)) + first_slice = await anext( + stream.stream_slices( + cursor_field=[stream.cursor_field] + if isinstance(stream.cursor_field, str) + else stream.cursor_field, + sync_mode=SyncMode.full_refresh, + ) + ) + return first_slice -async def get_first_record_for_slice(stream: AsyncStream, stream_slice: Optional[Mapping[str, Any]]) -> StreamData: +async def get_first_record_for_slice( + stream: AsyncStream, stream_slice: Optional[Mapping[str, Any]] +) -> AsyncGenerator[StreamData, None]: """ Gets the first record for a stream_slice of a stream. :param stream: stream @@ -29,5 +39,7 @@ async def get_first_record_for_slice(stream: AsyncStream, stream_slice: Optional """ # We wrap the return output of read_records() because some implementations return types that are iterable, # but not iterators such as lists or tuples - async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slice): + async for record in stream.read_records( + sync_mode=SyncMode.full_refresh, stream_slice=stream_slice + ): yield record diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py index 4cb90971b326..29663bb1d630 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py @@ -5,27 +5,44 @@ import logging import os -import urllib from abc import ABC, abstractmethod from pathlib import Path -from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Union -from urllib.parse import urljoin +from typing import ( + Any, + Callable, + Iterable, + List, + Mapping, + Optional, + Tuple, + Union, +) import requests import requests_cache from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy -from airbyte_cdk.sources.streams.call_rate import APIBudget, CachedLimiterSession, LimiterSession -from airbyte_cdk.sources.streams.core import Stream, StreamData -from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy +from airbyte_cdk.sources.streams.call_rate import ( + APIBudget, + CachedLimiterSession, + LimiterSession, +) +from airbyte_cdk.sources.streams.core import StreamData +from airbyte_cdk.sources.streams.http.availability_strategy import ( + HttpAvailabilityStrategy, +) from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream from airbyte_cdk.sources.utils.types import JsonType from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH from requests.auth import AuthBase from .auth.core import HttpAuthenticator, NoAuth -from .exceptions import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException +from .exceptions import ( + DefaultBackoffException, + RequestBodyException, + UserDefinedBackoffException, +) from .rate_limiting import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies @@ -37,15 +54,24 @@ class HttpStream(BaseHttpStream, ABC): Base abstract class for an Airbyte Stream using the HTTP protocol. Basic building block for users building an Airbyte source for a HTTP API. """ - source_defined_cursor = True # Most HTTP streams use a source defined cursor (i.e: the user can't configure it like on a SQL table) - page_size: Optional[int] = None # Use this variable to define page size for API http requests with pagination support + page_size: Optional[ + int + ] = None # Use this variable to define page size for API http requests with pagination support # TODO: remove legacy HttpAuthenticator authenticator references - def __init__(self, authenticator: Optional[Union[AuthBase, HttpAuthenticator]] = None, api_budget: Optional[APIBudget] = None): + def __init__( + self, + authenticator: Optional[Union[AuthBase, HttpAuthenticator]] = None, + api_budget: Optional[APIBudget] = None, + ): self._api_budget: APIBudget = api_budget or APIBudget(policies=[]) self._session = self.request_session() self._session.mount( - "https://", requests.adapters.HTTPAdapter(pool_connections=MAX_CONNECTION_POOL_SIZE, pool_maxsize=MAX_CONNECTION_POOL_SIZE) + "https://", + requests.adapters.HTTPAdapter( + pool_connections=MAX_CONNECTION_POOL_SIZE, + pool_maxsize=MAX_CONNECTION_POOL_SIZE, + ), ) self._authenticator: HttpAuthenticator = NoAuth() if isinstance(authenticator, AuthBase): @@ -86,7 +112,9 @@ def availability_strategy(self) -> Optional[AvailabilityStrategy]: return HttpAvailabilityStrategy() @abstractmethod - def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + def next_page_token( + self, response: requests.Response + ) -> Optional[Mapping[str, Any]]: """ Override this method to define a pagination strategy. @@ -173,7 +201,12 @@ def _create_prepared_request( query_params = self.deduplicate_query_params(url, params) else: query_params = params or {} - args = {"method": self.http_method, "url": url, "headers": headers, "params": query_params} + args = { + "method": self.http_method, + "url": url, + "headers": headers, + "params": query_params, + } if self.http_method.upper() in BODY_REQUEST_METHODS: if json and data: raise RequestBodyException( @@ -183,11 +216,15 @@ def _create_prepared_request( args["json"] = json elif data: args["data"] = data - prepared_request: requests.PreparedRequest = self._session.prepare_request(requests.Request(**args)) + prepared_request: requests.PreparedRequest = self._session.prepare_request( + requests.Request(**args) + ) return prepared_request - def _send(self, request: requests.PreparedRequest, request_kwargs: Mapping[str, Any]) -> requests.Response: + def _send( + self, request: requests.PreparedRequest, request_kwargs: Mapping[str, Any] + ) -> requests.Response: """ Wraps sending the request in rate limit and error handlers. Please note that error handling for HTTP status codes will be ignored if raise_on_http_errors is set to False @@ -207,7 +244,12 @@ def _send(self, request: requests.PreparedRequest, request_kwargs: Mapping[str, Unexpected persistent exceptions are not handled and will cause the sync to fail. """ self.logger.debug( - "Making outbound API request", extra={"headers": request.headers, "url": request.url, "request_body": request.body} + "Making outbound API request", + extra={ + "headers": request.headers, + "url": request.url, + "request_body": request.body, + }, ) response: requests.Response = self._session.send(request, **request_kwargs) @@ -215,17 +257,27 @@ def _send(self, request: requests.PreparedRequest, request_kwargs: Mapping[str, # Do it only in debug mode if self.logger.isEnabledFor(logging.DEBUG): self.logger.debug( - "Receiving response", extra={"headers": response.headers, "status": response.status_code, "body": response.text} + "Receiving response", + extra={ + "headers": response.headers, + "status": response.status_code, + "body": response.text, + }, ) if self.should_retry(response): custom_backoff_time = self.backoff_time(response) error_message = self.error_message(response) if custom_backoff_time: raise UserDefinedBackoffException( - backoff=custom_backoff_time, request=request, response=response, error_message=error_message + backoff=custom_backoff_time, + request=request, + response=response, + error_message=error_message, ) else: - raise DefaultBackoffException(request=request, response=response, error_message=error_message) + raise DefaultBackoffException( + request=request, response=response, error_message=error_message + ) elif self.raise_on_http_errors: # Raise any HTTP exceptions that happened in case there were unexpected ones try: @@ -235,7 +287,9 @@ def _send(self, request: requests.PreparedRequest, request_kwargs: Mapping[str, raise exc return response - def _send_request(self, request: requests.PreparedRequest, request_kwargs: Mapping[str, Any]) -> requests.Response: + def _send_request( + self, request: requests.PreparedRequest, request_kwargs: Mapping[str, Any] + ) -> requests.Response: """ Creates backoff wrappers which are responsible for retry logic """ @@ -271,8 +325,12 @@ def _send_request(self, request: requests.PreparedRequest, request_kwargs: Mappi if max_tries is not None: max_tries = max(0, max_tries) + 1 - user_backoff_handler = user_defined_backoff_handler(max_tries=max_tries, max_time=max_time)(self._send) - backoff_handler = default_backoff_handler(max_tries=max_tries, max_time=max_time, factor=self.retry_factor) + user_backoff_handler = user_defined_backoff_handler( + max_tries=max_tries, max_time=max_time + )(self._send) + backoff_handler = default_backoff_handler( + max_tries=max_tries, max_time=max_time, factor=self.retry_factor + ) return backoff_handler(user_backoff_handler)(request, request_kwargs) @classmethod @@ -334,13 +392,23 @@ def read_records( stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[StreamData]: yield from self._read_pages( - lambda req, res, state, _slice: self.parse_response(res, stream_slice=_slice, stream_state=state), stream_slice, stream_state + lambda req, res, state, _slice: self.parse_response( + res, stream_slice=_slice, stream_state=state + ), + stream_slice, + stream_state, ) def _read_pages( self, records_generator_fn: Callable[ - [requests.PreparedRequest, requests.Response, Mapping[str, Any], Optional[Mapping[str, Any]]], Iterable[StreamData] + [ + requests.PreparedRequest, + requests.Response, + Mapping[str, Any], + Optional[Mapping[str, Any]], + ], + Iterable[StreamData], ], stream_slice: Optional[Mapping[str, Any]] = None, stream_state: Optional[Mapping[str, Any]] = None, @@ -349,8 +417,12 @@ def _read_pages( pagination_complete = False next_page_token = None while not pagination_complete: - request, response = self._fetch_next_page(stream_slice, stream_state, next_page_token) - yield from records_generator_fn(request, response, stream_state, stream_slice) + request, response = self._fetch_next_page( + stream_slice, stream_state, next_page_token + ) + yield from records_generator_fn( + request, response, stream_state, stream_slice + ) next_page_token = self.next_page_token(response) if not next_page_token: @@ -365,15 +437,39 @@ def _fetch_next_page( stream_state: Optional[Mapping[str, Any]] = None, next_page_token: Optional[Mapping[str, Any]] = None, ) -> Tuple[requests.PreparedRequest, requests.Response]: - request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + request_headers = self.request_headers( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ) request = self._create_prepared_request( - path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + path=self.path( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), headers=dict(request_headers, **self.authenticator.get_auth_header()), - params=self.request_params(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), - json=self.request_body_json(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), - data=self.request_body_data(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + params=self.request_params( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + json=self.request_body_json( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + data=self.request_body_data( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, + ), + ) + request_kwargs = self.request_kwargs( + stream_state=stream_state, + stream_slice=stream_slice, + next_page_token=next_page_token, ) - request_kwargs = self.request_kwargs(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) response = self._send_request(request, request_kwargs) return request, response @@ -388,16 +484,24 @@ def __init__(self, parent: HttpStream, **kwargs: Any): self.parent = parent def stream_slices( - self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None + self, + sync_mode: SyncMode, + cursor_field: Optional[List[str]] = None, + stream_state: Optional[Mapping[str, Any]] = None, ) -> Iterable[Optional[Mapping[str, Any]]]: parent_stream_slices = self.parent.stream_slices( - sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_state=stream_state + sync_mode=SyncMode.full_refresh, + cursor_field=cursor_field, + stream_state=stream_state, ) # iterate over all parent stream_slices for stream_slice in parent_stream_slices: parent_records = self.parent.read_records( - sync_mode=SyncMode.full_refresh, cursor_field=cursor_field, stream_slice=stream_slice, stream_state=stream_state + sync_mode=SyncMode.full_refresh, + cursor_field=cursor_field, + stream_slice=stream_slice, + stream_state=stream_state, ) # iterate over all parent records with current stream_slice diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py index 2d59d3da0d69..c3606513e3c4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py @@ -22,8 +22,13 @@ class BaseHttpStream(Stream, ABC): Base abstract class for an Airbyte Stream using the HTTP protocol. Basic building block for users building an Airbyte source for a HTTP API. """ - source_defined_cursor = True # Most HTTP streams use a source defined cursor (i.e: the user can't configure it like on a SQL table) - page_size: Optional[int] = None # Use this variable to define page size for API http requests with pagination support + page_size: Optional[ + int + ] = None # Use this variable to define page size for API http requests with pagination support + + @property + def source_defined_cursor(self) -> bool: + return True @property def cache_filename(self) -> str: @@ -102,7 +107,9 @@ def authenticator(self) -> HttpAuthenticator: ... @abstractmethod - def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + def next_page_token( + self, response: requests.Response + ) -> Optional[Mapping[str, Any]]: """ Override this method to define a pagination strategy. @@ -245,7 +252,9 @@ def error_message(self, response: requests.Response) -> str: def must_deduplicate_query_params(self) -> bool: return False - def deduplicate_query_params(self, url: str, params: Optional[Mapping[str, Any]]) -> Mapping[str, Any]: + def deduplicate_query_params( + self, url: str, params: Optional[Mapping[str, Any]] + ) -> Mapping[str, Any]: """ Remove query parameters from params mapping if they are already encoded in the URL. :param url: URL with @@ -257,8 +266,12 @@ def deduplicate_query_params(self, url: str, params: Optional[Mapping[str, Any]] query_string = urllib.parse.urlparse(url).query query_dict = {k: v[0] for k, v in urllib.parse.parse_qs(query_string).items()} - duplicate_keys_with_same_value = {k for k in query_dict.keys() if str(params.get(k)) == str(query_dict[k])} - return {k: v for k, v in params.items() if k not in duplicate_keys_with_same_value} + duplicate_keys_with_same_value = { + k for k in query_dict.keys() if str(params.get(k)) == str(query_dict[k]) + } + return { + k: v for k, v in params.items() if k not in duplicate_keys_with_same_value + } @classmethod def _join_url(cls, url_base: str, path: str) -> str: From 8e3544ca202570bd1dcc52e569fa68fa79d54a80 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Sun, 7 Jan 2024 21:59:45 -0500 Subject: [PATCH 69/74] WIP --- .../sources/async_cdk/source_reader.py | 5 +- .../async_cdk/streams/http/http_async.py | 106 ++------------- .../airbyte_cdk/sources/streams/http/http.py | 38 +----- .../airbyte_cdk/sources/streams/http/utils.py | 123 ++++++++++++++++++ 4 files changed, 141 insertions(+), 131 deletions(-) create mode 100644 airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py index 9a1bc72b86e4..02882d9758c2 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_reader.py @@ -61,5 +61,6 @@ def drain(self): self.thread.join() async def cleanup(self): - for session in self.sessions.values(): - await session.close() + pass + # for session in self.sessions.values(): + # await session.close() diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py index 1c2d10915160..57333472f058 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py @@ -16,7 +16,6 @@ Mapping, Optional, Tuple, - TypeVar, Union, ) from yarl import URL @@ -42,17 +41,13 @@ from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.auth.core import HttpAuthenticator from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream -from airbyte_cdk.sources.utils.types import JsonType +from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH from .rate_limiting_async import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") -T1 = TypeVar("T1") -T2 = TypeVar("T2") -T3 = TypeVar("T3") -T4 = TypeVar("T4") RecordsGeneratorFunction = Callable[ [ aiohttp.ClientRequest, @@ -64,14 +59,6 @@ ] -""" - records_generator_fn: Callable[ - , - Iterable[StreamData], - ], -""" - - class AsyncHttpStream(BaseHttpStream, AsyncStream, ABC): """ Base abstract class for an Airbyte Stream using the HTTP protocol with asyncio. @@ -377,68 +364,6 @@ async def _send_request( ) return await backoff_handler(user_backoff_handler)(request, request_kwargs) - @classmethod - async def parse_response_error_message( - cls, exception: aiohttp.ClientResponseError - ) -> Optional[str]: - """ - Parses the raw response object from a failed request into a user-friendly error message. - By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. - - :param response: - :return: A user-friendly message that indicates the cause of the error - """ - - # default logic to grab error from common fields - def _try_get_error(value: Optional[JsonType]) -> Optional[str]: - if isinstance(value, str): - return value - elif isinstance(value, list): - errors_in_value = [_try_get_error(v) for v in value] - return ", ".join(v for v in errors_in_value if v is not None) - elif isinstance(value, dict): - new_value = ( - value.get("message") - or value.get("messages") - or value.get("error") - or value.get("errors") - or value.get("failures") - or value.get("failure") - or value.get("detail") - ) - return _try_get_error(new_value) - return None - - try: - if hasattr(exception, "_response_error"): - return _try_get_error(exception._response_error) # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 - else: - raise NotImplementedError( - "`_response_error` is expected but was not set on the response; `handle_response_with_error` should be used prior to processing the exception" - ) - except json.JSONDecodeError: - return None - - async def get_error_display_message( - self, exception: BaseException - ) -> Optional[str]: - """ - Retrieves the user-friendly display message that corresponds to an exception. - This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. - - The default implementation of this method only handles HTTPErrors by passing the response to self.parse_response_error_message(). - The method should be overriden as needed to handle any additional exception types. - - :param exception: The exception that was raised - :return: A user-friendly message that indicates the cause of the error - """ - if ( - isinstance(exception, aiohttp.ClientResponseError) - and exception.message is not None - ): - return await self.parse_response_error_message(exception) - return None - async def read_records( self, sync_mode: SyncMode, @@ -548,26 +473,19 @@ async def handle_response_with_error( """ if response.ok: return response - try: - error_json = await response.json() - except (json.JSONDecodeError, aiohttp.ContentTypeError): - error_json = None - except Exception as exc: - raise NotImplementedError(f"Unexpected!!!!!!!! {exc}") # TODO - self.logger.error(f"Unable to get error json from response: {exc}") - error_json = None - - exc = aiohttp.ClientResponseError( - response.request_info, - response.history, - status=response.status, - message=response.reason or "", - headers=response.headers, + + exc = HttpError( + aiohttp.ClientResponseError( + response.request_info, + response.history, + status=response.status, + message=response.reason or "", + headers=response.headers, + ), + response, ) + await exc.set_response_error() text = await response.text() - exc._response_error = ( # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 - error_json or text - ) self.logger.error(text) raise exc diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py index 29663bb1d630..9e27f8c9edc6 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py @@ -33,7 +33,7 @@ HttpAvailabilityStrategy, ) from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream -from airbyte_cdk.sources.utils.types import JsonType +from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH from requests.auth import AuthBase @@ -335,39 +335,7 @@ def _send_request( @classmethod def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: - """ - Parses the raw response object from a failed request into a user-friendly error message. - By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. - - :param response: - :return: A user-friendly message that indicates the cause of the error - """ - - # default logic to grab error from common fields - def _try_get_error(value: Optional[JsonType]) -> Optional[str]: - if isinstance(value, str): - return value - elif isinstance(value, list): - errors_in_value = [_try_get_error(v) for v in value] - return ", ".join(v for v in errors_in_value if v is not None) - elif isinstance(value, dict): - new_value = ( - value.get("message") - or value.get("messages") - or value.get("error") - or value.get("errors") - or value.get("failures") - or value.get("failure") - or value.get("detail") - ) - return _try_get_error(new_value) - return None - - try: - body = response.json() - return _try_get_error(body) - except requests.exceptions.JSONDecodeError: - return None + return HttpError.parse_response_error_message(response) def get_error_display_message(self, exception: BaseException) -> Optional[str]: """ @@ -380,7 +348,7 @@ def get_error_display_message(self, exception: BaseException) -> Optional[str]: :param exception: The exception that was raised :return: A user-friendly message that indicates the cause of the error """ - if isinstance(exception, requests.HTTPError) and exception.response is not None: + if isinstance(exception, HttpError) and exception.response is not None: return self.parse_response_error_message(exception.response) return None diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py new file mode 100644 index 000000000000..9ab6255b3224 --- /dev/null +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py @@ -0,0 +1,123 @@ +import json +from typing import Optional, Union + +import aiohttp +import requests + +from airbyte_cdk.sources.utils.types import JsonType + + +class HttpError(Exception): + def __init__( + self, + error: Union[requests.HTTPError, aiohttp.ClientResponseError], + response: Optional[aiohttp.ClientResponse] = None, + ): + self.error = error + + self.response = ( + response if isinstance(response, aiohttp.ClientResponse) else error.response + ) + self._response_error: Optional[JsonType] = None + + @property + def status_code(self) -> Optional[int]: + if isinstance(self.error, requests.HTTPError): + return self.error.response.status_code if self.error.response else None + elif isinstance(self.error, aiohttp.ClientResponseError): + return self.error.status + + @property + def message(self) -> str: + if isinstance(self.error, requests.HTTPError): + return str(self.error) + elif isinstance(self.error, aiohttp.ClientResponseError): + return self.error.message + + @property + def error_response(self) -> JsonType: + # This returns a uniform response object for further introspection + if isinstance(self.error, requests.HTTPError): + return self.error.response.json() + elif isinstance(self.error, aiohttp.ClientResponseError): + return self._response_error + + @classmethod + def parse_response_error_message( + cls, response: Union[requests.Response, aiohttp.ClientResponseError] + ) -> Optional[str]: + if isinstance(response, requests.Response): + return cls._parse_sync_response_error(response) + else: + return cls._parse_async_response_error(response) + + @classmethod + def _parse_sync_response_error(cls, response: requests.Response) -> Optional[str]: + """ + Parses the raw response object from a failed request into a user-friendly error message. + By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. + + :param response: + :return: A user-friendly message that indicates the cause of the error + """ + try: + return cls._try_get_error(response.json()) + except requests.exceptions.JSONDecodeError: + return None + + @classmethod + def _parse_async_response_error( + cls, response: aiohttp.ClientResponseError + ) -> Optional[str]: + try: + if ( + hasattr(response, "_response_error") + and response._response_error is not None + ): + return cls._try_get_error(response) # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 + else: + raise NotImplementedError( + "`_response_error` is expected but was not set on the response; `set_response_error` should be used prior to processing the exception" + ) + except json.JSONDecodeError: + return None + + @classmethod + def _try_get_error(cls, value: Optional[JsonType]) -> Optional[str]: + # default logic to grab error from common fields + if isinstance(value, str): + return value + elif isinstance(value, list): + errors_in_value = [cls._try_get_error(v) for v in value] + return ", ".join(v for v in errors_in_value if v is not None) + elif isinstance(value, dict): + new_value = ( + value.get("message") + or value.get("messages") + or value.get("error") + or value.get("errors") + or value.get("failures") + or value.get("failure") + or value.get("detail") + ) + return cls._try_get_error(new_value) + return None + + # Async utils + + async def set_response_error(self): + if self._response is None: + return + try: + error_json = await self._response.json() + except (json.JSONDecodeError, aiohttp.ContentTypeError): + error_json = None + except Exception as exc: + raise NotImplementedError(f"Unexpected!!!!!!!! {exc}") # TODO + self.logger.error(f"Unable to get error json from response: {exc}") + error_json = None + + text = await self._response.text() + exception._response_error = ( # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 + error_json or text + ) From 63a379c41cec65c67bde9392958ced859d953991 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Mon, 8 Jan 2024 13:01:52 -0500 Subject: [PATCH 70/74] Make unified sync/async error --- .../sources/async_cdk/source_dispatcher.py | 2 +- .../sources/async_cdk/streams/core_async.py | 2 +- .../http/availability_strategy_async.py | 27 +- .../streams/http/exceptions_async.py | 22 +- .../async_cdk/streams/http/http_async.py | 47 +-- .../streams/http/rate_limiting_async.py | 28 +- .../streams/http/availability_strategy.py | 45 ++- .../airbyte_cdk/sources/streams/http/http.py | 15 - .../sources/streams/http/http_base.py | 9 +- .../sources/streams/http/rate_limiting.py | 28 +- .../airbyte_cdk/sources/streams/http/utils.py | 133 ++++---- .../http/test_availability_strategy_async.py | 45 ++- .../async_cdk/streams/http/test_http_async.py | 304 ++++++++++++++---- .../async_cdk/test_abstract_source_async.py | 2 +- .../source_salesforce/source.py | 9 +- .../source_salesforce/streams.py | 31 +- .../source-salesforce/unit_tests/api_test.py | 4 +- 17 files changed, 506 insertions(+), 247 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py index 83618b290e5d..c79b4cb3c813 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py @@ -292,7 +292,7 @@ async def _async_read_stream( configured_stream.stream, AirbyteStreamStatus.INCOMPLETE ) ) - display_message = await stream_instance.get_error_display_message(e) + display_message = stream_instance.get_error_display_message(e) if display_message: raise AirbyteTracedException.from_exception(e, message=display_message) else: diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py index 791135410763..583bde479210 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py @@ -186,7 +186,7 @@ async def check_availability( ) return True, None - async def get_error_display_message( + def get_error_display_message( self, exception: BaseException ) -> Optional[str]: """ diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py index 468925bcd30a..811a02a148cb 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py @@ -3,12 +3,10 @@ # import logging -from typing import Dict, Optional, Tuple +from typing import TYPE_CHECKING, Dict, Optional, Tuple import requests -from aiohttp import ClientResponseError from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource -from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream from airbyte_cdk.sources.async_cdk.streams.utils.stream_helper_async import ( get_first_record_for_slice, get_first_stream_slice, @@ -16,12 +14,16 @@ from airbyte_cdk.sources.streams.http.availability_strategy import ( HttpAvailabilityStrategy, ) +from airbyte_cdk.sources.streams.http.utils import HttpError + +if TYPE_CHECKING: + from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream class AsyncHttpAvailabilityStrategy(HttpAvailabilityStrategy): async def check_availability( self, - stream: AsyncStream, + stream: "AsyncHttpStream", logger: logging.Logger, source: Optional["AsyncAbstractSource"], ) -> Tuple[bool, Optional[str]]: @@ -47,7 +49,7 @@ async def check_availability( # without accounting for the case in which the parent stream is empty. reason = f"Cannot attempt to connect to stream {stream.name} - no stream slices were found, likely because the parent stream is empty." return False, reason - except ClientResponseError as error: + except HttpError as error: is_available, reason = await self._handle_http_error( stream, logger, source, error ) @@ -63,7 +65,7 @@ async def check_availability( f"Successfully connected to stream {stream.name}, but got 0 records." ) return True, None - except ClientResponseError as error: + except HttpError as error: is_available, reason = await self._handle_http_error( stream, logger, source, error ) @@ -75,10 +77,10 @@ async def check_availability( async def _handle_http_error( self, - stream: AsyncStream, + stream: "AsyncHttpStream", logger: logging.Logger, source: Optional["AsyncAbstractSource"], - error: ClientResponseError, + error: HttpError, ) -> Tuple[bool, Optional[str]]: """ Override this method to define error handling for various `HTTPError`s @@ -96,7 +98,7 @@ async def _handle_http_error( for some reason and the str should describe what went wrong and how to resolve the unavailability, if possible. """ - status_code = error.status + status_code = error.status_code known_status_codes = self.reasons_for_unavailable_status_codes( stream, logger, source, error ) @@ -106,15 +108,16 @@ async def _handle_http_error( raise error doc_ref = self._visit_docs_message(logger, source) - reason = f"The endpoint {error.request_info.url} returned {status_code}: {error.message}. {known_reason}. {doc_ref} " + reason = f"The endpoint {error.url} returned {status_code}: {error.message}. {known_reason}. {doc_ref} " + response_error_message = stream.parse_error_message(error) return False, reason def reasons_for_unavailable_status_codes( self, - stream: AsyncStream, + stream: "AsyncHttpStream", logger: logging.Logger, source: Optional["AsyncAbstractSource"], - error: ClientResponseError, + error: HttpError, ) -> Dict[int, str]: """ Returns a dictionary of HTTP status codes that indicate stream diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py index ea5228fab2b5..9b26d0834a7c 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py @@ -7,13 +7,16 @@ import aiohttp +from airbyte_cdk.sources.streams.http.utils import HttpError -class BaseBackoffException(aiohttp.ClientResponseError): - def __init__(self, response: aiohttp.ClientResponse, error_message: str = ""): - error_message = ( - error_message or f"Request URL: {response.request_info.url}, Response Code: {response.status}, Response Text: {response.text}" + +class BaseBackoffException(HttpError): + def __init__(self, error: HttpError, error_message: str = ""): + error._aiohttp_error.message = ( + error_message + or f"Request URL: {error.url}, Response Code: {error.status_code}, Response Text: {error.text}" ) - super().__init__(request_info=response.request_info, history=(response,), status=response.status, message=error_message, headers=response.headers) + super().__init__(aiohttp_error=error._aiohttp_error) # TODO class RequestBodyException(Exception): @@ -27,14 +30,19 @@ class UserDefinedBackoffException(BaseBackoffException): An exception that exposes how long it attempted to backoff """ - def __init__(self, backoff: Union[int, float], response: aiohttp.ClientResponse, error_message: str = ""): + def __init__( + self, + backoff: Union[int, float], + error: HttpError, + error_message: str = "", + ): """ :param backoff: how long to backoff in seconds :param request: the request that triggered this backoff exception :param response: the response that triggered the backoff exception """ self.backoff = backoff - super().__init__(response=response, error_message=error_message) + super().__init__(error, error_message=error_message) class DefaultBackoffException(BaseBackoffException): diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py index 57333472f058..30f0b2dc2414 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py @@ -22,6 +22,7 @@ import aiohttp import aiohttp_client_cache +import requests from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.async_cdk.streams.async_call_rate import ( AsyncCachedLimiterSession, @@ -295,23 +296,26 @@ async def _send( "body": response.text, }, ) - if self.should_retry(response): - custom_backoff_time = self.backoff_time(response) - error_message = self.error_message(response) - if custom_backoff_time: - raise UserDefinedBackoffException( - backoff=custom_backoff_time, - response=response, - error_message=error_message, - ) - else: - raise DefaultBackoffException( - response=response, error_message=error_message - ) - elif self.raise_on_http_errors: - # Raise any HTTP exceptions that happened in case there were unexpected ones + try: return await self.handle_response_with_error(response) - return response + except HttpError as exc: + if self.should_retry(response): + custom_backoff_time = self.backoff_time(response) + error_message = self.error_message(response) + if custom_backoff_time: + raise UserDefinedBackoffException( + backoff=custom_backoff_time, + error=exc, + error_message=error_message, + ) + else: + raise DefaultBackoffException( + error=exc, error_message=error_message + ) + elif self.raise_on_http_errors: + # Raise any HTTP exceptions that happened in case there were unexpected ones + raise exc + return response async def ensure_session(self) -> aiohttp.ClientSession: if self._session is None: @@ -364,6 +368,12 @@ async def _send_request( ) return await backoff_handler(user_backoff_handler)(request, request_kwargs) + @classmethod + def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: + raise NotImplementedError( + "Async streams should use HttpError.parse_error_message" + ) + async def read_records( self, sync_mode: SyncMode, @@ -475,16 +485,15 @@ async def handle_response_with_error( return response exc = HttpError( - aiohttp.ClientResponseError( + aiohttp_error=aiohttp.ClientResponseError( response.request_info, response.history, status=response.status, message=response.reason or "", headers=response.headers, ), - response, ) - await exc.set_response_error() + await exc.set_response_data(response) text = await response.text() self.logger.error(text) raise exc diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py index c5bd5f3816be..9e70037b66ef 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py @@ -10,6 +10,7 @@ import aiohttp import backoff +from airbyte_cdk.sources.streams.http.utils import HttpError from .exceptions_async import DefaultBackoffException, UserDefinedBackoffException TRANSIENT_EXCEPTIONS = ( @@ -43,21 +44,21 @@ def default_backoff_handler( ) -> Callable[[SendRequestCallableType], SendRequestCallableType]: def log_retry_attempt(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() - if isinstance(exc, aiohttp.ClientResponseError) and exc.history: + if isinstance(exc, HttpError): logger.info( - f"Status code: {exc.status}, Response Content: {'FIX ME'}" - ) # TODO + f"Status code: {exc.status_code}, Response Content: {exc.content}" + ) + message = exc.message if hasattr(exc, "message") else type(exc) logger.info( - f"Caught retryable error '{str(exc)}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." + f"Caught retryable error '{message}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." ) def should_give_up(exc: Exception) -> bool: # If a non-rate-limiting related 4XX error makes it this far, it means it was unexpected and probably consistent, so we shouldn't back off - if isinstance(exc, aiohttp.ClientResponseError): + if isinstance(exc, HttpError): give_up: bool = ( - exc.history is not None - and exc.status != TOO_MANY_REQUESTS_CODE - and 400 <= exc.status < 500 + exc.status_code != TOO_MANY_REQUESTS_CODE + and 400 <= exc.status_code < 500 ) if give_up: logger.info(f"Giving up for returned HTTP status: {exc.status}") @@ -84,19 +85,18 @@ def user_defined_backoff_handler( def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, UserDefinedBackoffException): - if exc.history: - logger.info( - f"Status code: {exc.status}, Response Content: {'FIX ME'}" # TODO - ) # TODO + logger.info( + f"Status code: {exc.status_code}, Response Content: {exc.content}" + ) retry_after = exc.backoff logger.info(f"Retrying. Sleeping for {retry_after} seconds") time.sleep(retry_after + 1) # extra second to cover any fractions of second def log_give_up(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() - if isinstance(exc, aiohttp.ClientResponseError): + if isinstance(exc, HttpError): logger.error( - f"Max retry limit reached. Request: {exc.request_info}, Response: {exc.history}" + f"Max retry limit reached. Request: {exc.url}, Response: {exc.content}" ) # TODO: how does history get printed out else: logger.error("Max retry limit reached for unknown request and response") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py index 3f8755070c4b..7da175c0ae94 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py @@ -9,7 +9,10 @@ import requests from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy -from airbyte_cdk.sources.streams.utils.stream_helper import get_first_record_for_slice, get_first_stream_slice +from airbyte_cdk.sources.streams.utils.stream_helper import ( + get_first_record_for_slice, + get_first_stream_slice, +) from requests import HTTPError if typing.TYPE_CHECKING: @@ -17,7 +20,9 @@ class HttpAvailabilityStrategy(AvailabilityStrategy): - def check_availability(self, stream: Stream, logger: logging.Logger, source: Optional["Source"]) -> Tuple[bool, Optional[str]]: + def check_availability( + self, stream: Stream, logger: logging.Logger, source: Optional["Source"] + ) -> Tuple[bool, Optional[str]]: """ Check stream availability by attempting to read the first record of the stream. @@ -41,7 +46,7 @@ def check_availability(self, stream: Stream, logger: logging.Logger, source: Opt reason = f"Cannot attempt to connect to stream {stream.name} - no stream slices were found, likely because the parent stream is empty." return False, reason except HTTPError as error: - is_available, reason = self.handle_http_error(stream, logger, source, error) + is_available, reason = self._handle_http_error(stream, logger, source, error) if not is_available: reason = f"Unable to get slices for {stream.name} stream, because of error in parent stream. {reason}" return is_available, reason @@ -50,16 +55,22 @@ def check_availability(self, stream: Stream, logger: logging.Logger, source: Opt get_first_record_for_slice(stream, stream_slice) return True, None except StopIteration: - logger.info(f"Successfully connected to stream {stream.name}, but got 0 records.") + logger.info( + f"Successfully connected to stream {stream.name}, but got 0 records." + ) return True, None except HTTPError as error: - is_available, reason = self.handle_http_error(stream, logger, source, error) + is_available, reason = self._handle_http_error(stream, logger, source, error) if not is_available: reason = f"Unable to read {stream.name} stream. {reason}" return is_available, reason - def handle_http_error( - self, stream: Stream, logger: logging.Logger, source: Optional["Source"], error: HTTPError + def _handle_http_error( + self, + stream: Stream, + logger: logging.Logger, + source: Optional["Source"], + error: HTTPError, ) -> Tuple[bool, Optional[str]]: """ Override this method to define error handling for various `HTTPError`s @@ -78,7 +89,9 @@ def handle_http_error( resolve the unavailability, if possible. """ status_code = error.response.status_code - known_status_codes = self.reasons_for_unavailable_status_codes(stream, logger, source, error) + known_status_codes = self.reasons_for_unavailable_status_codes( + stream, logger, source, error + ) known_reason = known_status_codes.get(status_code) if not known_reason: # If the HTTPError is not in the dictionary of errors we know how to handle, don't except @@ -86,13 +99,17 @@ def handle_http_error( doc_ref = self._visit_docs_message(logger, source) reason = f"The endpoint {error.response.url} returned {status_code}: {error.response.reason}. {known_reason}. {doc_ref} " - response_error_message = stream.parse_response_error_message(error.response) + response_error_message = stream.parse_error_message(error) if response_error_message: reason += response_error_message return False, reason def reasons_for_unavailable_status_codes( - self, stream: Stream, logger: logging.Logger, source: Optional["Source"], error: HTTPError + self, + stream: Stream, + logger: logging.Logger, + source: Optional["Source"], + error: HTTPError, ) -> Dict[int, str]: """ Returns a dictionary of HTTP status codes that indicate stream @@ -134,9 +151,13 @@ def _visit_docs_message(logger: logging.Logger, source: Optional["Source"]) -> s else: return "Please visit the connector's documentation to learn more. " - except FileNotFoundError: # If we are unit testing without implementing spec() method in source + except ( + FileNotFoundError + ): # If we are unit testing without implementing spec() method in source if source: - docs_url = f"https://docs.airbyte.com/integrations/sources/{source.name}" + docs_url = ( + f"https://docs.airbyte.com/integrations/sources/{source.name}" + ) else: docs_url = "https://docs.airbyte.com/integrations/sources/test" diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py index 9e27f8c9edc6..775ad77bab55 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py @@ -337,21 +337,6 @@ def _send_request( def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: return HttpError.parse_response_error_message(response) - def get_error_display_message(self, exception: BaseException) -> Optional[str]: - """ - Retrieves the user-friendly display message that corresponds to an exception. - This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. - - The default implementation of this method only handles HTTPErrors by passing the response to self.parse_response_error_message(). - The method should be overriden as needed to handle any additional exception types. - - :param exception: The exception that was raised - :return: A user-friendly message that indicates the cause of the error - """ - if isinstance(exception, HttpError) and exception.response is not None: - return self.parse_response_error_message(exception.response) - return None - def read_records( self, sync_mode: SyncMode, diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py index c3606513e3c4..7b749ab3356a 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py @@ -10,6 +10,7 @@ import requests from airbyte_cdk.models import SyncMode from airbyte_cdk.sources.streams.core import Stream, StreamData +from airbyte_cdk.sources.streams.http.utils import HttpError from .auth.core import HttpAuthenticator @@ -288,7 +289,10 @@ def parse_response_error_message(cls, response: requests.Response) -> Optional[s :return: A user-friendly message that indicates the cause of the error """ - @abstractmethod + @classmethod + def parse_error_message(cls, error: HttpError) -> Optional[str]: + return error.parse_error_message() + def get_error_display_message(self, exception: BaseException) -> Optional[str]: """ Retrieves the user-friendly display message that corresponds to an exception. @@ -300,6 +304,9 @@ def get_error_display_message(self, exception: BaseException) -> Optional[str]: :param exception: The exception that was raised :return: A user-friendly message that indicates the cause of the error """ + if isinstance(exception, HttpError): + return self.parse_error_message(exception) + return None @abstractmethod def read_records( diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py index 84d320345294..a45f9791cdbe 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py @@ -10,6 +10,7 @@ import backoff from requests import PreparedRequest, RequestException, Response, codes, exceptions +from airbyte_cdk.sources.streams.http.utils import HttpError from .exceptions import DefaultBackoffException, UserDefinedBackoffException TRANSIENT_EXCEPTIONS = ( @@ -27,12 +28,17 @@ def default_backoff_handler( - max_tries: Optional[int], factor: float, max_time: Optional[int] = None, **kwargs: Any + max_tries: Optional[int], + factor: float, + max_time: Optional[int] = None, + **kwargs: Any, ) -> Callable[[SendRequestCallableType], SendRequestCallableType]: def log_retry_attempt(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() - if isinstance(exc, RequestException) and exc.response: - logger.info(f"Status code: {exc.response.status_code}, Response Content: {exc.response.content}") + if isinstance(exc, HttpError): + logger.info( + f"Status code: {exc.status_code}, Response Content: {exc.content}" + ) logger.info( f"Caught retryable error '{str(exc)}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." ) @@ -41,10 +47,14 @@ def should_give_up(exc: Exception) -> bool: # If a non-rate-limiting related 4XX error makes it this far, it means it was unexpected and probably consistent, so we shouldn't back off if isinstance(exc, RequestException): give_up: bool = ( - exc.response is not None and exc.response.status_code != codes.too_many_requests and 400 <= exc.response.status_code < 500 + exc.response is not None + and exc.response.status_code != codes.too_many_requests + and 400 <= exc.response.status_code < 500 ) if give_up: - logger.info(f"Giving up for returned HTTP status: {exc.response.status_code}") + logger.info( + f"Giving up for returned HTTP status: {exc.response.status_code}" + ) return give_up # Only RequestExceptions are retryable, so if we get here, it's not retryable return False @@ -69,7 +79,9 @@ def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, UserDefinedBackoffException): if exc.response: - logger.info(f"Status code: {exc.response.status_code}, Response Content: {exc.response.content}") + logger.info( + f"Status code: {exc.response.status_code}, Response Content: {exc.response.content}" + ) retry_after = exc.backoff logger.info(f"Retrying. Sleeping for {retry_after} seconds") time.sleep(retry_after + 1) # extra second to cover any fractions of second @@ -77,7 +89,9 @@ def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: def log_give_up(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, RequestException): - logger.error(f"Max retry limit reached. Request: {exc.request}, Response: {exc.response}") + logger.error( + f"Max retry limit reached. Request: {exc.request}, Response: {exc.response}" + ) else: logger.error("Max retry limit reached for unknown request and response") diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py index 9ab6255b3224..888e67a53e92 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py @@ -1,5 +1,5 @@ import json -from typing import Optional, Union +from typing import Optional import aiohttp import requests @@ -10,49 +10,68 @@ class HttpError(Exception): def __init__( self, - error: Union[requests.HTTPError, aiohttp.ClientResponseError], - response: Optional[aiohttp.ClientResponse] = None, + requests_error: Optional[requests.HTTPError] = None, + aiohttp_error: Optional[aiohttp.ClientResponseError] = None, ): - self.error = error - - self.response = ( - response if isinstance(response, aiohttp.ClientResponse) else error.response - ) - self._response_error: Optional[JsonType] = None + assert ( + requests_error or aiohttp_error and not (requests_error and aiohttp_error) + ), "requests_error xor aiohttp_error must be supplied" + self._requests_error = requests_error + self._aiohttp_error = aiohttp_error + self._aiohttp_response_json = None + self._aiohttp_response_content = None + self._aiohttp_response_text = None @property def status_code(self) -> Optional[int]: - if isinstance(self.error, requests.HTTPError): - return self.error.response.status_code if self.error.response else None - elif isinstance(self.error, aiohttp.ClientResponseError): - return self.error.status + if self._requests_error and self._requests_error.response: + return self._requests_error.response.status_code + elif self._aiohttp_error: + return self._aiohttp_error.status + return 0 @property def message(self) -> str: - if isinstance(self.error, requests.HTTPError): - return str(self.error) - elif isinstance(self.error, aiohttp.ClientResponseError): - return self.error.message + if self._requests_error: + return str(self._requests_error) + elif self._aiohttp_error: + return self._aiohttp_error.message + else: + return "" @property - def error_response(self) -> JsonType: - # This returns a uniform response object for further introspection - if isinstance(self.error, requests.HTTPError): - return self.error.response.json() - elif isinstance(self.error, aiohttp.ClientResponseError): - return self._response_error + def content(self) -> Optional[bytes]: + if self._requests_error and self._requests_error.response: + return self._requests_error.response.content + elif self._aiohttp_error: + return self._aiohttp_response_content + return b"" - @classmethod - def parse_response_error_message( - cls, response: Union[requests.Response, aiohttp.ClientResponseError] - ) -> Optional[str]: - if isinstance(response, requests.Response): - return cls._parse_sync_response_error(response) - else: - return cls._parse_async_response_error(response) + @property + def text(self) -> Optional[str]: + if self._requests_error and self._requests_error.response: + return self._requests_error.response.text + elif self._aiohttp_error: + return self._aiohttp_response_text + return "" + + def json(self) -> Optional[JsonType]: + if self._requests_error and self._requests_error.response: + return self._requests_error.response.json() + elif self._aiohttp_error: + return self._aiohttp_response_json + return "" + + @property + def url(self) -> str: + if self._requests_error and self._requests_error.request: + return self._requests_error.request.url or "" + elif self._aiohttp_error: + return str(self._aiohttp_error.request_info.url) + return "" @classmethod - def _parse_sync_response_error(cls, response: requests.Response) -> Optional[str]: + def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: """ Parses the raw response object from a failed request into a user-friendly error message. By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. @@ -65,22 +84,22 @@ def _parse_sync_response_error(cls, response: requests.Response) -> Optional[str except requests.exceptions.JSONDecodeError: return None - @classmethod - def _parse_async_response_error( - cls, response: aiohttp.ClientResponseError - ) -> Optional[str]: - try: - if ( - hasattr(response, "_response_error") - and response._response_error is not None - ): - return cls._try_get_error(response) # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 - else: - raise NotImplementedError( - "`_response_error` is expected but was not set on the response; `set_response_error` should be used prior to processing the exception" - ) - except json.JSONDecodeError: - return None + def parse_error_message(self) -> Optional[str]: + """ + Parses the raw response object from a failed request into a user-friendly error message. + By default, this method tries to grab the error message from JSON responses by following common API patterns. Override to parse differently. + + :param response: + :return: A user-friendly message that indicates the cause of the error + """ + if self._requests_error and self._requests_error.response: + return self.parse_response_error_message(self._requests_error.response) + elif self._aiohttp_error: + try: + return self._try_get_error(self._aiohttp_response_json) + except requests.exceptions.JSONDecodeError: + return None + return None @classmethod def _try_get_error(cls, value: Optional[JsonType]) -> Optional[str]: @@ -105,19 +124,17 @@ def _try_get_error(cls, value: Optional[JsonType]) -> Optional[str]: # Async utils - async def set_response_error(self): - if self._response is None: - return + async def set_response_data(self, response: aiohttp.ClientResponse): try: - error_json = await self._response.json() + response_json = await response.json() except (json.JSONDecodeError, aiohttp.ContentTypeError): - error_json = None + response_json = None except Exception as exc: raise NotImplementedError(f"Unexpected!!!!!!!! {exc}") # TODO self.logger.error(f"Unable to get error json from response: {exc}") - error_json = None + response_json = None - text = await self._response.text() - exception._response_error = ( # type: ignore # https://github.com/aio-libs/aiohttp/issues/3248 - error_json or text - ) + text = await response.text() # This fixed a test + self._aiohttp_response_json = response_json or text + self._aiohttp_response_content = await response.content.read() + self._aiohttp_response_text = text diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py index deb50d2b0c6b..830e62f2284a 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_availability_strategy_async.py @@ -10,9 +10,12 @@ from aioresponses import aioresponses from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.async_cdk.source_dispatcher import SourceDispatcher -from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy +from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import ( + AsyncHttpAvailabilityStrategy, +) from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.http.utils import HttpError logger = logging.getLogger("airbyte") @@ -25,13 +28,17 @@ def __init__(self, **kwargs): super().__init__(**kwargs) self.resp_counter = 1 - async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + async def next_page_token( + self, response: aiohttp.ClientResponse + ) -> Optional[Mapping[str, Any]]: return None def path(self, **kwargs) -> str: return "" - async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: + async def parse_response( + self, response: aiohttp.ClientResponse, **kwargs + ) -> Iterable[Mapping]: stub_resp = {"data": self.resp_counter} self.resp_counter += 1 yield stub_resp @@ -57,7 +64,12 @@ def retry_factor(self) -> float: @pytest.mark.parametrize( ("include_source", "expected_docs_url_messages"), [ - (True, ["Please visit https://docs.airbyte.com/integrations/sources/MockSource to learn more."]), + ( + True, + [ + "Please visit https://docs.airbyte.com/integrations/sources/MockSource to learn more." + ], + ), (False, ["Please visit the connector's documentation to learn more."]), ], ) @@ -81,7 +93,9 @@ def __init__(self, streams: List[Stream] = None): self._streams = streams super().__init__() - async def check_connection(self, logger: logging.Logger, config: Mapping[str, Any]) -> Tuple[bool, Optional[Any]]: + async def check_connection( + self, logger: logging.Logger, config: Mapping[str, Any] + ) -> Tuple[bool, Optional[Any]]: return True, "" async def streams(self, config: Mapping[str, Any]) -> List[Stream]: @@ -96,7 +110,9 @@ async def streams(self, config: Mapping[str, Any]) -> List[Stream]: if include_source: source = SourceDispatcher(MockSource(streams=[http_stream])) - actual_is_available, reason = await http_stream.check_availability(logger, source.async_source) + actual_is_available, reason = await http_stream.check_availability( + logger, source.async_source + ) else: actual_is_available, reason = await http_stream.check_availability(logger) @@ -121,7 +137,7 @@ def test_http_availability_raises_unhandled_error(mocker): with aioresponses() as m: m.get(http_stream.url_base, status=404) - with pytest.raises(aiohttp.ClientResponseError): + with pytest.raises(HttpError): loop.run_until_complete(http_stream.check_availability(logger)) @@ -144,16 +160,23 @@ def request_callback(*args, **kwargs): m.get(http_stream.url_base, status=200, callback=request_callback) with caplog.at_level(logging.INFO): - stream_is_available, _ = loop.run_until_complete(http_stream.check_availability(logger)) + stream_is_available, _ = loop.run_until_complete( + http_stream.check_availability(logger) + ) assert stream_is_available assert call_counter == 3 - for message in ["Caught retryable error", "Response Code: 429", "Response Code: 503"]: + for message in [ + "Caught retryable error", + "Response Code: 429", + "Response Code: 503", + ]: assert message in caplog.text def test_http_availability_strategy_on_empty_stream(mocker): empty_stream_called = False + async def empty_aiter(*args, **kwargs): nonlocal empty_stream_called empty_stream_called = True @@ -170,7 +193,9 @@ def __init__(self, *args, **kvargs): logger = logging.getLogger("airbyte.test-source") loop = asyncio.get_event_loop() - stream_is_available, _ = loop.run_until_complete(empty_stream.check_availability(logger)) + stream_is_available, _ = loop.run_until_complete( + empty_stream.check_availability(logger) + ) assert stream_is_available assert empty_stream_called diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py index f246a210c0e9..08c6e051cbb1 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py @@ -7,17 +7,27 @@ import json from http import HTTPStatus from typing import Any, Iterable, Mapping, Optional -from unittest.mock import MagicMock, patch +from unittest.mock import AsyncMock, MagicMock, patch from yarl import URL import aiohttp import pytest from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import SyncMode -from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream -from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import DefaultBackoffException, RequestBodyException, UserDefinedBackoffException +from airbyte_cdk.sources.async_cdk.streams.http.http_async import ( + AsyncHttpStream, + AsyncHttpSubStream, +) +from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import ( + DefaultBackoffException, + RequestBodyException, + UserDefinedBackoffException, +) from airbyte_cdk.sources.streams.http.auth import NoAuth -from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator as HttpTokenAuthenticator +from airbyte_cdk.sources.streams.http.auth import ( + TokenAuthenticator as HttpTokenAuthenticator, +) +from airbyte_cdk.sources.streams.http.utils import HttpError class StubBasicReadHttpStream(AsyncHttpStream): @@ -29,13 +39,17 @@ def __init__(self, deduplicate_query_params: bool = False, **kwargs): self.resp_counter = 1 self._deduplicate_query_params = deduplicate_query_params - async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + async def next_page_token( + self, response: aiohttp.ClientResponse + ) -> Optional[Mapping[str, Any]]: return None def path(self, **kwargs) -> str: return "" - async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: + async def parse_response( + self, response: aiohttp.ClientResponse, **kwargs + ) -> Iterable[Mapping]: stubResp = {"data": self.resp_counter} self.resp_counter += 1 yield stubResp @@ -68,12 +82,16 @@ def test_request_kwargs_used(mocker): m.assert_any_call(stream.url_base, "GET", **request_kwargs) m.assert_called_once() - loop.run_until_complete(stream._session.close()) # TODO - find a way to not manually close after each test + loop.run_until_complete( + stream._session.close() + ) # TODO - find a way to not manually close after each test async def read_records(stream, sync_mode=SyncMode.full_refresh, stream_slice=None): records = [] - async for record in stream.read_records(sync_mode=sync_mode, stream_slice=stream_slice): + async for record in stream.read_records( + sync_mode=sync_mode, stream_slice=stream_slice + ): records.append(record) return records @@ -81,8 +99,12 @@ async def read_records(stream, sync_mode=SyncMode.full_refresh, stream_slice=Non def test_stub_basic_read_http_stream_read_records(mocker): loop = asyncio.get_event_loop() stream = StubBasicReadHttpStream() - blank_response = {} # Send a blank response is fine as we ignore the response in `parse_response anyway. - mocker.patch.object(StubBasicReadHttpStream, "_send_request", return_value=blank_response) + blank_response = ( + {} + ) # Send a blank response is fine as we ignore the response in `parse_response anyway. + mocker.patch.object( + StubBasicReadHttpStream, "_send_request", return_value=blank_response + ) records = loop.run_until_complete(read_records(stream)) @@ -96,7 +118,9 @@ def __init__(self, pages: int = 5): super().__init__() self._pages = pages - async def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + async def next_page_token( + self, response: aiohttp.ClientResponse + ) -> Optional[Mapping[str, Any]]: while self.current_page < self._pages: page_token = {"page": self.current_page} self.current_page += 1 @@ -108,8 +132,12 @@ def test_next_page_token_is_input_to_other_methods(mocker): """Validates that the return value from next_page_token is passed into other methods that need it like request_params, headers, body, etc..""" pages = 5 stream = StubNextPageTokenHttpStream(pages=pages) - blank_response = {} # Send a blank response is fine as we ignore the response in `parse_response anyway. - mocker.patch.object(StubNextPageTokenHttpStream, "_send_request", return_value=blank_response) + blank_response = ( + {} + ) # Send a blank response is fine as we ignore the response in `parse_response anyway. + mocker.patch.object( + StubNextPageTokenHttpStream, "_send_request", return_value=blank_response + ) methods = ["request_params", "request_headers", "request_body_json"] for method in methods: @@ -123,12 +151,23 @@ def test_next_page_token_is_input_to_other_methods(mocker): expected_next_page_tokens = [{"page": i} for i in range(pages)] for method in methods: # First assert that they were called with no next_page_token. This is the first call in the pagination loop. - getattr(stream, method).assert_any_call(next_page_token=None, stream_slice=None, stream_state={}) + getattr(stream, method).assert_any_call( + next_page_token=None, stream_slice=None, stream_state={} + ) for token in expected_next_page_tokens: # Then verify that each method - getattr(stream, method).assert_any_call(next_page_token=token, stream_slice=None, stream_state={}) - - expected = [{"data": 1}, {"data": 2}, {"data": 3}, {"data": 4}, {"data": 5}, {"data": 6}] + getattr(stream, method).assert_any_call( + next_page_token=token, stream_slice=None, stream_state={} + ) + + expected = [ + {"data": 1}, + {"data": 2}, + {"data": 3}, + {"data": 4}, + {"data": 5}, + {"data": 6}, + ] assert expected == records @@ -218,6 +257,7 @@ def max_retries(self): call_counter = 0 with aioresponses() as m: + def request_callback(*args, **kwargs): nonlocal call_counter call_counter += 1 @@ -226,7 +266,12 @@ def request_callback(*args, **kwargs): # to stop the infinite retries raise RuntimeError("End of retries") - m.get(stream.url_base, status=HTTPStatus.TOO_MANY_REQUESTS, repeat=True, callback=request_callback) + m.get( + stream.url_base, + status=HTTPStatus.TOO_MANY_REQUESTS, + repeat=True, + callback=request_callback, + ) # Expecting mock object to raise a RuntimeError when the end of side_effect list parameter reached. with pytest.raises(RuntimeError): @@ -245,7 +290,7 @@ def test_4xx_error_codes_http_stream(http_code): with aioresponses() as m: m.get(stream.url_base, status=http_code, repeat=True) - with pytest.raises(aiohttp.ClientResponseError): + with pytest.raises(HttpError): loop.run_until_complete(read_records(stream)) loop.run_until_complete(stream._session.close()) @@ -282,7 +327,9 @@ def request_callback(*args, **kwargs): call_counter += 1 with aioresponses() as m: - m.get(stream.url_base, status=status_code, repeat=True, callback=request_callback) + m.get( + stream.url_base, status=status_code, repeat=True, callback=request_callback + ) with pytest.raises(DefaultBackoffException): loop.run_until_complete(read_records(stream)) @@ -298,7 +345,9 @@ def test_raise_on_http_errors_off_non_retryable_4xx(status_code): with aioresponses() as m: m.get(stream.url_base, status=status_code, repeat=True) - response = loop.run_until_complete(stream._send_request(aiohttp.ClientRequest("GET", URL(stream.url_base)), {})) + response = loop.run_until_complete( + stream._send_request(aiohttp.ClientRequest("GET", URL(stream.url_base)), {}) + ) assert response.status == status_code loop.run_until_complete(stream._session.close()) @@ -324,7 +373,9 @@ def request_callback(*args, **kwargs): call_counter += 1 with aioresponses() as m: - m.get(stream.url_base, repeat=True, callback=request_callback, exception=error()) + m.get( + stream.url_base, repeat=True, callback=request_callback, exception=error() + ) with pytest.raises(error): loop.run_until_complete(read_records(stream)) @@ -336,7 +387,9 @@ def request_callback(*args, **kwargs): class PostHttpStream(StubBasicReadHttpStream): http_method = "POST" - async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: + async def parse_response( + self, response: aiohttp.ClientResponse, **kwargs + ) -> Iterable[Mapping]: """Returns response data as is""" yield response.json() @@ -355,7 +408,7 @@ def request2response(self, **kwargs): headers = kwargs.get("headers", {}) return { "body": json.dumps(body) if isinstance(body, dict) else body, - "content_type": headers.get("Content-Type") + "content_type": headers.get("Content-Type"), } def test_json_body(self, mocker): @@ -365,7 +418,12 @@ def test_json_body(self, mocker): loop.run_until_complete(stream.ensure_session()) with aioresponses() as m: - m.post(stream.url_base, payload=self.request2response(data=self.json_body, headers={"Content-Type": "application/json"})) + m.post( + stream.url_base, + payload=self.request2response( + data=self.json_body, headers={"Content-Type": "application/json"} + ), + ) response = [] for r in loop.run_until_complete(read_records(stream)): @@ -427,7 +485,9 @@ def test_body_for_all_methods(self, mocker, requests_mock): } for method, with_body in methods.items(): stream.http_method = method - mocker.patch.object(stream, "request_body_data", return_value=self.data_body) + mocker.patch.object( + stream, "request_body_data", return_value=self.data_body + ) with aioresponses() as m: if method == "POST": @@ -443,7 +503,9 @@ def test_body_for_all_methods(self, mocker, requests_mock): elif method == "OPTIONS": request = m.options - request(stream.url_base, payload=self.request2response(data=self.data_body)) + request( + stream.url_base, payload=self.request2response(data=self.data_body) + ) response = [] for r in loop.run_until_complete(read_records(stream)): @@ -467,10 +529,14 @@ class CacheHttpSubStream(AsyncHttpSubStream): def __init__(self, parent): super().__init__(parent=parent) - async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: + async def parse_response( + self, response: aiohttp.ClientResponse, **kwargs + ) -> Iterable[Mapping]: yield None - def next_page_token(self, response: aiohttp.ClientResponse) -> Optional[Mapping[str, Any]]: + def next_page_token( + self, response: aiohttp.ClientResponse + ) -> Optional[Mapping[str, Any]]: return None def path(self, **kwargs) -> str: @@ -534,7 +600,9 @@ async def stream_slices(self, **kwargs) -> Iterable[Optional[Mapping[str, Any]]] for path in self.paths: yield {"path": path} - async def parse_response(self, response: aiohttp.ClientResponse, **kwargs) -> Iterable[Mapping]: + async def parse_response( + self, response: aiohttp.ClientResponse, **kwargs + ) -> Iterable[Mapping]: yield {"value": len(await response.text())} @@ -554,6 +622,7 @@ def test_using_cache(mocker): mocker.patch.object(parent_stream, "url_base", "https://google.com/") call_counter = 0 + def request_callback(*args, **kwargs): nonlocal call_counter call_counter += 1 @@ -566,7 +635,9 @@ async def get_urls(stream): with aioresponses() as m: # Set up the mocks - slices = loop.run_until_complete(stream_slices(parent_stream, sync_mode=SyncMode.full_refresh)) + slices = loop.run_until_complete( + stream_slices(parent_stream, sync_mode=SyncMode.full_refresh) + ) m.get(parent_stream.url_base, callback=request_callback) m.get(f"{parent_stream.url_base}search", callback=request_callback) @@ -586,7 +657,9 @@ async def get_urls(stream): # child_stream.stream_slices will call `parent.read_records`, however this shouldn't # result in a new request to the 3rd party since the response has been cached - loop.run_until_complete(stream_slices(child_stream, sync_mode=SyncMode.full_refresh)) + loop.run_until_complete( + stream_slices(child_stream, sync_mode=SyncMode.full_refresh) + ) assert call_counter == 2 urls = loop.run_until_complete(get_urls(parent_stream)) @@ -605,7 +678,9 @@ class AutoFailTrueHttpStream(StubBasicReadHttpStream): @pytest.mark.parametrize("status_code", range(400, 600)) def test_send_raise_on_http_errors_logs(mocker, status_code): mocker.patch.object(AutoFailTrueHttpStream, "logger") - mocker.patch.object(AutoFailTrueHttpStream, "should_retry", mocker.Mock(return_value=False)) + mocker.patch.object( + AutoFailTrueHttpStream, "should_retry", mocker.Mock(return_value=False) + ) stream = AutoFailTrueHttpStream() loop = asyncio.get_event_loop() @@ -616,7 +691,7 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): with aioresponses() as m: m.get(stream.url_base, status=status_code, repeat=True, payload="text") - with pytest.raises(aiohttp.ClientResponseError): + with pytest.raises(HttpError): response = loop.run_until_complete(stream._send_request(req, {})) stream.logger.error.assert_called_with("text") assert response.status == status_code @@ -632,13 +707,42 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): ({"error": {"message": "something broke"}}, "something broke"), ({"error": "err-001", "message": "something broke"}, "something broke"), ({"failure": {"message": "something broke"}}, "something broke"), - ({"error": {"errors": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}}, "one, two, three"), + ( + { + "error": { + "errors": [ + {"message": "one"}, + {"message": "two"}, + {"message": "three"}, + ] + } + }, + "one, two, three", + ), ({"errors": ["one", "two", "three"]}, "one, two, three"), ({"messages": ["one", "two", "three"]}, "one, two, three"), - ({"errors": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, "one, two, three"), - ({"error": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, "one, two, three"), - ({"errors": [{"error": "one"}, {"error": "two"}, {"error": "three"}]}, "one, two, three"), - ({"failures": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, "one, two, three"), + ( + {"errors": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, + "one, two, three", + ), + ( + {"error": [{"message": "one"}, {"message": "two"}, {"message": "three"}]}, + "one, two, three", + ), + ( + {"errors": [{"error": "one"}, {"error": "two"}, {"error": "three"}]}, + "one, two, three", + ), + ( + { + "failures": [ + {"message": "one"}, + {"message": "two"}, + {"message": "three"}, + ] + }, + "one, two, three", + ), (["one", "two", "three"], "one, two, three"), ([{"error": "one"}, {"error": "two"}, {"error": "three"}], "one, two, three"), ({"error": True}, None), @@ -646,13 +750,31 @@ def test_send_raise_on_http_errors_logs(mocker, status_code): ({}, None), ], ) -async def test_default_parse_response_error_message(api_response: dict, expected_message: Optional[str]): +async def test_default_parse_response_error_message( + api_response: dict, expected_message: Optional[str] +): stream = StubBasicReadHttpStream() response = MagicMock() - response._response_error = api_response + response.ok = False + response.json = AsyncMock(return_value=api_response) + response.text = AsyncMock() + response.content = AsyncMock() + exc_message = None + try: + await stream.handle_response_with_error(response) + except HttpError as exc: + exc_message = stream.get_error_display_message(exc) + + assert exc_message == expected_message + +""" + stream = StubBasicReadHttpStream() + response = MagicMock() + response.json.return_value = api_response - message = await stream.parse_response_error_message(response) + message = stream.parse_response_error_message(response) assert message == expected_message +""" @pytest.mark.asyncio @@ -662,9 +784,16 @@ async def test_default_parse_response_error_message_not_json(): url = "mock://test.com/not_json" with aioresponses() as m: - m.get(url, callback=lambda *_, **__: CallbackResult(status=400, body="this is not json")) - with pytest.raises(aiohttp.ClientResponseError): - response = await stream._send_request(aiohttp.ClientRequest("GET", URL(url)), {}) + m.get( + url, + callback=lambda *_, **__: CallbackResult( + status=400, body="this is not json" + ), + ) + with pytest.raises(HttpError): + response = await stream._send_request( + aiohttp.ClientRequest("GET", URL(url)), {} + ) message = await stream.parse_response_error_message(response) assert message is None await stream._session.close() @@ -672,41 +801,66 @@ async def test_default_parse_response_error_message_not_json(): def test_default_get_error_display_message_handles_http_error(mocker): stream = StubBasicReadHttpStream() - loop = asyncio.get_event_loop() - loop.run_until_complete(stream.ensure_session()) - mocker.patch.object(stream, "parse_response_error_message", return_value="my custom message") + mocker.patch.object( + stream, "parse_error_message", return_value="my custom message" + ) - non_http_err_msg = loop.run_until_complete(stream.get_error_display_message(RuntimeError("not me"))) + non_http_err_msg = stream.get_error_display_message(RuntimeError("not me")) assert non_http_err_msg is None - req = aiohttp.ClientRequest("GET", URL("mock://test.com/not_json")) - - with aioresponses() as m: - m.get("mock://test.com/not_json") - response = loop.run_until_complete(stream._send_request(req, {})) - http_exception = aiohttp.ClientResponseError(request_info=None, history=None, message=response) + req = aiohttp.ClientRequest("GET", URL(stream.url_base)) - http_err_msg = loop.run_until_complete(stream.get_error_display_message(http_exception)) + error = HttpError(aiohttp_error=aiohttp.ClientResponseError(request_info=req.request_info, history=(), status=400, message="", headers={})) + http_err_msg = stream.get_error_display_message(error) assert http_err_msg == "my custom message" - loop.run_until_complete(stream._session.close()) @pytest.mark.parametrize( "test_name, base_url, path, expected_full_url", [ - ("test_no_slashes", "https://airbyte.io", "my_endpoint", "https://airbyte.io/my_endpoint"), - ("test_trailing_slash_on_base_url", "https://airbyte.io/", "my_endpoint", "https://airbyte.io/my_endpoint"), + ( + "test_no_slashes", + "https://airbyte.io", + "my_endpoint", + "https://airbyte.io/my_endpoint", + ), + ( + "test_trailing_slash_on_base_url", + "https://airbyte.io/", + "my_endpoint", + "https://airbyte.io/my_endpoint", + ), ( "test_trailing_slash_on_base_url_and_leading_slash_on_path", "https://airbyte.io/", "/my_endpoint", "https://airbyte.io/my_endpoint", ), - ("test_leading_slash_on_path", "https://airbyte.io", "/my_endpoint", "https://airbyte.io/my_endpoint"), - ("test_trailing_slash_on_path", "https://airbyte.io", "/my_endpoint/", "https://airbyte.io/my_endpoint/"), - ("test_nested_path_no_leading_slash", "https://airbyte.io", "v1/my_endpoint", "https://airbyte.io/v1/my_endpoint"), - ("test_nested_path_with_leading_slash", "https://airbyte.io", "/v1/my_endpoint", "https://airbyte.io/v1/my_endpoint"), + ( + "test_leading_slash_on_path", + "https://airbyte.io", + "/my_endpoint", + "https://airbyte.io/my_endpoint", + ), + ( + "test_trailing_slash_on_path", + "https://airbyte.io", + "/my_endpoint/", + "https://airbyte.io/my_endpoint/", + ), + ( + "test_nested_path_no_leading_slash", + "https://airbyte.io", + "v1/my_endpoint", + "https://airbyte.io/v1/my_endpoint", + ), + ( + "test_nested_path_with_leading_slash", + "https://airbyte.io", + "/v1/my_endpoint", + "https://airbyte.io/v1/my_endpoint", + ), ], ) def test_join_url(test_name, base_url, path, expected_full_url): @@ -718,12 +872,26 @@ def test_join_url(test_name, base_url, path, expected_full_url): "deduplicate_query_params, path, params, expected_url", [ pytest.param( - True, "v1/endpoint?param1=value1", {}, "https://test_base_url.com/v1/endpoint?param1=value1", id="test_params_only_in_path" + True, + "v1/endpoint?param1=value1", + {}, + "https://test_base_url.com/v1/endpoint?param1=value1", + id="test_params_only_in_path", ), pytest.param( - True, "v1/endpoint", {"param1": "value1"}, "https://test_base_url.com/v1/endpoint?param1=value1", id="test_params_only_in_path" + True, + "v1/endpoint", + {"param1": "value1"}, + "https://test_base_url.com/v1/endpoint?param1=value1", + id="test_params_only_in_path", + ), + pytest.param( + True, + "v1/endpoint", + None, + "https://test_base_url.com/v1/endpoint", + id="test_params_is_none_and_no_params_in_path", ), - pytest.param(True, "v1/endpoint", None, "https://test_base_url.com/v1/endpoint", id="test_params_is_none_and_no_params_in_path"), pytest.param( True, "v1/endpoint?param1=value1", @@ -775,7 +943,9 @@ def test_join_url(test_name, base_url, path, expected_full_url): ), ], ) -def test_duplicate_request_params_are_deduped(deduplicate_query_params, path, params, expected_url): +def test_duplicate_request_params_are_deduped( + deduplicate_query_params, path, params, expected_url +): stream = StubBasicReadHttpStream(deduplicate_query_params) if expected_url is None: diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py index f0842460d480..06b9d6b38b6c 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/test_abstract_source_async.py @@ -326,7 +326,7 @@ def test_read_stream_with_error_gets_display_message(mocker): with pytest.raises(RuntimeError, match="oh no!"): list(source.read(logger, {}, catalog)) - stream.get_error_display_message = AsyncMock(return_value="my message") + mocker.patch.object(MockStream, "get_error_display_message", return_value="my message") with pytest.raises(AirbyteTracedException, match="oh no!") as exc: list(source.read(logger, {}, catalog)) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py index 706a051ea99c..ec12286faabb 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -17,6 +17,7 @@ from airbyte_cdk.sources.message import InMemoryMessageRepository from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator +from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.sources.utils.schema_helpers import InternalConfig from airbyte_cdk.utils.traced_exception import AirbyteTracedException from dateutil.relativedelta import relativedelta @@ -212,11 +213,11 @@ async def read_stream( try: async for record in super().read_stream(logger, stream_instance, configured_stream, state_manager, internal_config): yield record - except aiohttp.ClientResponseError as error: - error_data = error._response_error if hasattr(error, "_response_error") else {} + except HttpError as error: + error_data = error.json() error_code = error_data.get("errorCode") - url = error.request_info.url - if error.status == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + url = error.url + if error.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": logger.warning(f"API Call {url} limit is exceeded. Error message: '{error_data.get('message')}'") raise AirbyteStopSync() # if got 403 rate limit response, finish the sync with success. raise error diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py index f711f78b6812..56917aab5386 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -19,10 +19,11 @@ import pandas as pd import pendulum import requests # type: ignore[import] -from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream from airbyte_cdk.models import ConfiguredAirbyteCatalog, FailureType, SyncMode +from airbyte_cdk.sources.async_cdk.streams.http.http_async import AsyncHttpStream, AsyncHttpSubStream from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy from airbyte_cdk.sources.streams.core import Stream, StreamData +from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.sources.utils.transform import TransformConfig, TypeTransformer from airbyte_cdk.utils import AirbyteTracedException from numpy import nan @@ -107,10 +108,10 @@ def get_json_schema(self) -> Mapping[str, Any]: self.schema = self.sf_api.generate_schema(self.name) return self.schema - async def get_error_display_message(self, exception: BaseException) -> Optional[str]: - if isinstance(exception, aiohttp.ClientResponseError): + def get_error_display_message(self, exception: BaseException) -> Optional[str]: + if isinstance(exception, HttpError): return f"After {self.max_retries} retries the connector has failed with a network error. It looks like Salesforce API experienced temporary instability, please try again later." - return await super().get_error_display_message(exception) + return super().get_error_display_message(exception) class PropertyChunk: @@ -369,8 +370,8 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: response = await self._send_http_request("POST", url, json=json) job_id: str = (await response.json())["id"] return job_id - except aiohttp.ClientResponseError as error: # TODO: which errors? - if error.status in [codes.FORBIDDEN, codes.BAD_REQUEST]: + except HttpError as error: # TODO: which errors? + if error.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: # A part of streams can't be used by BULK API. Every API version can have a custom list of # these sobjects. Another part of them can be generated dynamically. That's why we can't track # them preliminarily and there is only one way is to except error with necessary messages about @@ -383,9 +384,7 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: # updated query: "Select Name, (Select Subject,ActivityType from ActivityHistories) from Contact" # The second variant forces customisation for every case (ActivityHistory, ActivityHistories etc). # And the main problem is these subqueries doesn't support CSV response format. - if not hasattr(error, "_response_error"): - raise NotImplementedError("!!!!!!!!!!!!! this didn't use `handle_response_with_error`") - error_data = error._response_error or {} + error_data = error.json() or {} error_code = error_data.get("errorCode") error_message = error_data.get("message", "") if error_message == "Selecting compound data not supported in Bulk Query" or ( @@ -395,29 +394,29 @@ async def create_stream_job(self, query: str, url: str) -> Optional[str]: f"Cannot receive data for stream '{self.name}' using BULK API, " f"sobject options: {self.sobject_options}, error message: '{error_message}'" ) - elif error.status == codes.FORBIDDEN and error_code != "REQUEST_LIMIT_EXCEEDED": + elif error.status_code == codes.FORBIDDEN and error_code != "REQUEST_LIMIT_EXCEEDED": self.logger.error( f"Cannot receive data for stream '{self.name}' ," f"sobject options: {self.sobject_options}, error message: '{error_message}'" ) - elif error.status == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + elif error.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": self.logger.error( f"Cannot receive data for stream '{self.name}' ," f"sobject options: {self.sobject_options}, Error message: '{error_data.get('message')}'" ) - elif error.status == codes.BAD_REQUEST and error_message.endswith("does not support query"): + elif error.status_code == codes.BAD_REQUEST and error_message.endswith("does not support query"): self.logger.error( f"The stream '{self.name}' is not queryable, " f"sobject options: {self.sobject_options}, error message: '{error_message}'" ) elif ( - error.status == codes.BAD_REQUEST + error.status_code == codes.BAD_REQUEST and error_code == "API_ERROR" and error_message.startswith("Implementation restriction") ): message = f"Unable to sync '{self.name}'. To prevent future syncs from failing, ensure the authenticated user has \"View all Data\" permissions." raise AirbyteTracedException(message=message, failure_type=FailureType.config_error, exception=error) - elif error.status == codes.BAD_REQUEST and error_code == "LIMIT_EXCEEDED": + elif error.status_code == codes.BAD_REQUEST and error_code == "LIMIT_EXCEEDED": message = "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed." self.logger.error(message) else: @@ -438,8 +437,8 @@ async def wait_for_job(self, url: str) -> str: while pendulum.now() < expiration_time: try: job_info = await (await self._send_http_request("GET", url=url)).json() - except aiohttp.ClientResponseError as error: - error_data = error._response_error + except HttpError as error: + error_data = error.json() error_code = error_data.get("errorCode") error_message = error_data.get("message", "") if ( diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index 43084f71f77f..d323dd0a024e 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -11,7 +11,6 @@ from unittest.mock import Mock from yarl import URL -import aiohttp import freezegun import pendulum import pytest @@ -21,6 +20,7 @@ from airbyte_cdk.sources.async_cdk import source_dispatcher from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade +from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils import AirbyteTracedException from conftest import encoding_symbols_parameters, generate_stream from source_salesforce.api import Salesforce @@ -91,7 +91,7 @@ def callback(*args, **kwargs): with aioresponses() as m: m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) - with pytest.raises(aiohttp.ClientResponseError) as err: + with pytest.raises(HttpError) as err: stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] assert err.value.message == "test_error" From 9c7bd9cea71ff3ddcb0e1b445be16ac4efc61c32 Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Mon, 8 Jan 2024 20:32:22 -0500 Subject: [PATCH 71/74] Remove some repetetive rate limiting code --- .../sources/async_cdk/streams/core_async.py | 22 +--- .../http/availability_strategy_async.py | 75 +----------- .../streams/http/exceptions_async.py | 19 +-- .../async_cdk/streams/http/http_async.py | 14 +-- .../streams/http/rate_limiting_async.py | 114 ------------------ .../streams/utils/stream_helper_async.py | 18 ++- .../streams/http/availability_strategy.py | 18 ++- .../sources/streams/http/exceptions.py | 7 +- .../airbyte_cdk/sources/streams/http/http.py | 3 + .../sources/streams/http/http_base.py | 3 + .../sources/streams/http/rate_limiting.py | 84 +++++++++---- .../airbyte_cdk/sources/streams/http/utils.py | 32 ++++- .../async_cdk/streams/http/test_http_async.py | 14 +-- .../streams/utils/test_rate_limiting_async.py | 26 ---- .../http/test_availability_strategy.py | 2 +- .../unit_tests/utils/test_rate_limiting.py | 5 + .../source-salesforce/unit_tests/api_test.py | 5 +- 17 files changed, 157 insertions(+), 304 deletions(-) delete mode 100644 airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py delete mode 100644 airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py index 583bde479210..d1ba29a14770 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/core_async.py @@ -113,15 +113,11 @@ async def read_incremental( # type: ignore # ignoring typing for ConnectorStat if internal_config.is_limit_reached(record_counter): break - yield self._checkpoint_state( - stream_state, state_manager, per_stream_state_enabled - ) + yield self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) if not has_slices: # Safety net to ensure we always emit at least one state message even if there are no slices - checkpoint = self._checkpoint_state( - stream_state, state_manager, per_stream_state_enabled - ) + checkpoint = self._checkpoint_state(stream_state, state_manager, per_stream_state_enabled) yield checkpoint @abstractmethod @@ -185,17 +181,3 @@ async def check_availability( self, logger, source ) return True, None - - def get_error_display_message( - self, exception: BaseException - ) -> Optional[str]: - """ - Retrieves the user-friendly display message that corresponds to an exception. - This will be called when encountering an exception while reading records from the stream, and used to build the AirbyteTraceMessage. - - The default implementation of this method does not return user-friendly messages for any exception type, but it should be overriden as needed. - - :param exception: The exception that was raised - :return: A user-friendly message that indicates the cause of the error - """ - return None diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py index 811a02a148cb..5f428b0e3bb4 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/availability_strategy_async.py @@ -3,9 +3,8 @@ # import logging -from typing import TYPE_CHECKING, Dict, Optional, Tuple +from typing import TYPE_CHECKING, Optional, Tuple -import requests from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.async_cdk.streams.utils.stream_helper_async import ( get_first_record_for_slice, @@ -50,23 +49,21 @@ async def check_availability( reason = f"Cannot attempt to connect to stream {stream.name} - no stream slices were found, likely because the parent stream is empty." return False, reason except HttpError as error: - is_available, reason = await self._handle_http_error( - stream, logger, source, error - ) + is_available, reason = self._handle_http_error(stream, logger, source, error) if not is_available: reason = f"Unable to get slices for {stream.name} stream, because of error in parent stream. {reason}" return is_available, reason try: - async for _ in get_first_record_for_slice(stream, stream_slice): - return True, None + await get_first_record_for_slice(stream, stream_slice) + return True, None except StopAsyncIteration: logger.info( f"Successfully connected to stream {stream.name}, but got 0 records." ) return True, None except HttpError as error: - is_available, reason = await self._handle_http_error( + is_available, reason = self._handle_http_error( stream, logger, source, error ) if not is_available: @@ -74,65 +71,3 @@ async def check_availability( return is_available, reason return True, None - - async def _handle_http_error( - self, - stream: "AsyncHttpStream", - logger: logging.Logger, - source: Optional["AsyncAbstractSource"], - error: HttpError, - ) -> Tuple[bool, Optional[str]]: - """ - Override this method to define error handling for various `HTTPError`s - that are raised while attempting to check a stream's availability. - - Checks whether an error's status_code is in a list of unavailable_error_codes, - and gets the associated reason for that error. - - :param stream: stream - :param logger: source logger - :param source: optional (source) - :param error: HTTPError raised while checking stream's availability. - :return: A tuple of (boolean, str). If boolean is true, then the stream - is available, and no str is required. Otherwise, the stream is unavailable - for some reason and the str should describe what went wrong and how to - resolve the unavailability, if possible. - """ - status_code = error.status_code - known_status_codes = self.reasons_for_unavailable_status_codes( - stream, logger, source, error - ) - known_reason = known_status_codes.get(status_code) - if not known_reason: - # If the HTTPError is not in the dictionary of errors we know how to handle, don't except - raise error - - doc_ref = self._visit_docs_message(logger, source) - reason = f"The endpoint {error.url} returned {status_code}: {error.message}. {known_reason}. {doc_ref} " - response_error_message = stream.parse_error_message(error) - return False, reason - - def reasons_for_unavailable_status_codes( - self, - stream: "AsyncHttpStream", - logger: logging.Logger, - source: Optional["AsyncAbstractSource"], - error: HttpError, - ) -> Dict[int, str]: - """ - Returns a dictionary of HTTP status codes that indicate stream - unavailability and reasons explaining why a given status code may - have occurred and how the user can resolve that error, if applicable. - - :param stream: stream - :param logger: source logger - :param source: optional (source) - :return: A dictionary of (status code, reason) where the 'reason' explains - why 'status code' may have occurred and how the user can resolve that - error, if applicable. - """ - reasons_for_codes: Dict[int, str] = { - requests.codes.FORBIDDEN: "This is most likely due to insufficient permissions on the credentials in use. " - "Try to grant required permissions/scopes or re-authenticate" - } - return reasons_for_codes diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py index 9b26d0834a7c..f0c08bd85f95 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/exceptions_async.py @@ -5,27 +5,20 @@ from typing import Union -import aiohttp - from airbyte_cdk.sources.streams.http.utils import HttpError +from airbyte_cdk.sources.streams.http.exceptions import AbstractBaseBackoffException -class BaseBackoffException(HttpError): +class AsyncBaseBackoffException(AbstractBaseBackoffException, HttpError): def __init__(self, error: HttpError, error_message: str = ""): - error._aiohttp_error.message = ( + error_message = ( error_message or f"Request URL: {error.url}, Response Code: {error.status_code}, Response Text: {error.text}" ) - super().__init__(aiohttp_error=error._aiohttp_error) # TODO - - -class RequestBodyException(Exception): - """ - Raised when there are issues in configuring a request body - """ + super().__init__(aiohttp_error=error._aiohttp_error, error_message=error_message) -class UserDefinedBackoffException(BaseBackoffException): +class AsyncUserDefinedBackoffException(AsyncBaseBackoffException): """ An exception that exposes how long it attempted to backoff """ @@ -45,5 +38,5 @@ def __init__( super().__init__(error, error_message=error_message) -class DefaultBackoffException(BaseBackoffException): +class AsyncDefaultBackoffException(AsyncBaseBackoffException): pass diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py index 30f0b2dc2414..01343b6d8f7d 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/http_async.py @@ -33,19 +33,19 @@ AsyncHttpAvailabilityStrategy, ) from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import ( - DefaultBackoffException, - RequestBodyException, - UserDefinedBackoffException, + AsyncDefaultBackoffException, + AsyncUserDefinedBackoffException, ) from airbyte_cdk.sources.http_config import MAX_CONNECTION_POOL_SIZE from airbyte_cdk.sources.streams.core import StreamData from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.auth.core import HttpAuthenticator +from airbyte_cdk.sources.streams.http.exceptions import RequestBodyException from airbyte_cdk.sources.streams.http.http_base import BaseHttpStream +from airbyte_cdk.sources.streams.http.rate_limiting import default_backoff_handler, async_user_defined_backoff_handler from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils.constants import ENV_REQUEST_CACHE_PATH -from .rate_limiting_async import default_backoff_handler, user_defined_backoff_handler # list of all possible HTTP methods which can be used for sending of request bodies BODY_REQUEST_METHODS = ("GET", "POST", "PUT", "PATCH") @@ -303,13 +303,13 @@ async def _send( custom_backoff_time = self.backoff_time(response) error_message = self.error_message(response) if custom_backoff_time: - raise UserDefinedBackoffException( + raise AsyncUserDefinedBackoffException( backoff=custom_backoff_time, error=exc, error_message=error_message, ) else: - raise DefaultBackoffException( + raise AsyncDefaultBackoffException( error=exc, error_message=error_message ) elif self.raise_on_http_errors: @@ -360,7 +360,7 @@ async def _send_request( if max_tries is not None: max_tries = max(0, max_tries) + 1 - user_backoff_handler = user_defined_backoff_handler( + user_backoff_handler = async_user_defined_backoff_handler( max_tries=max_tries, max_time=max_time )(self._send) backoff_handler = default_backoff_handler( diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py deleted file mode 100644 index 9e70037b66ef..000000000000 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/http/rate_limiting_async.py +++ /dev/null @@ -1,114 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -import logging -import sys -import time -from typing import Any, Callable, Coroutine, Mapping, Optional - -import aiohttp -import backoff - -from airbyte_cdk.sources.streams.http.utils import HttpError -from .exceptions_async import DefaultBackoffException, UserDefinedBackoffException - -TRANSIENT_EXCEPTIONS = ( - DefaultBackoffException, - # TODO - # exceptions.ConnectTimeout, - # exceptions.ReadTimeout, - # exceptions.ConnectionError, - # exceptions.ChunkedEncodingError, - aiohttp.ClientPayloadError, - aiohttp.ServerTimeoutError, - aiohttp.ServerConnectionError, - aiohttp.ServerDisconnectedError, -) - -logger = logging.getLogger("airbyte") - - -SendRequestCallableType = Callable[ - [aiohttp.ClientRequest, Mapping[str, Any]], - Coroutine[Any, Any, aiohttp.ClientResponse], -] -TOO_MANY_REQUESTS_CODE = 429 - - -def default_backoff_handler( - max_tries: Optional[int], - factor: float, - max_time: Optional[int] = None, - **kwargs: Any, -) -> Callable[[SendRequestCallableType], SendRequestCallableType]: - def log_retry_attempt(details: Mapping[str, Any]) -> None: - _, exc, _ = sys.exc_info() - if isinstance(exc, HttpError): - logger.info( - f"Status code: {exc.status_code}, Response Content: {exc.content}" - ) - message = exc.message if hasattr(exc, "message") else type(exc) - logger.info( - f"Caught retryable error '{message}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." - ) - - def should_give_up(exc: Exception) -> bool: - # If a non-rate-limiting related 4XX error makes it this far, it means it was unexpected and probably consistent, so we shouldn't back off - if isinstance(exc, HttpError): - give_up: bool = ( - exc.status_code != TOO_MANY_REQUESTS_CODE - and 400 <= exc.status_code < 500 - ) - if give_up: - logger.info(f"Giving up for returned HTTP status: {exc.status}") - return give_up - # Only RequestExceptions are retryable, so if we get here, it's not retryable - return False - - return backoff.on_exception( - backoff.expo, - TRANSIENT_EXCEPTIONS, - jitter=None, - on_backoff=log_retry_attempt, - giveup=should_give_up, - max_tries=max_tries, - max_time=max_time, - factor=factor, - **kwargs, - ) - - -def user_defined_backoff_handler( - max_tries: Optional[int], max_time: Optional[int] = None, **kwargs: Any -) -> Callable[[SendRequestCallableType], SendRequestCallableType]: - def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: - _, exc, _ = sys.exc_info() - if isinstance(exc, UserDefinedBackoffException): - logger.info( - f"Status code: {exc.status_code}, Response Content: {exc.content}" - ) - retry_after = exc.backoff - logger.info(f"Retrying. Sleeping for {retry_after} seconds") - time.sleep(retry_after + 1) # extra second to cover any fractions of second - - def log_give_up(details: Mapping[str, Any]) -> None: - _, exc, _ = sys.exc_info() - if isinstance(exc, HttpError): - logger.error( - f"Max retry limit reached. Request: {exc.url}, Response: {exc.content}" - ) # TODO: how does history get printed out - else: - logger.error("Max retry limit reached for unknown request and response") - - return backoff.on_exception( - backoff.constant, - UserDefinedBackoffException, - interval=0, # skip waiting, we'll wait in on_backoff handler - on_backoff=sleep_on_ratelimit, - on_giveup=log_give_up, - jitter=None, - max_tries=max_tries, - max_time=max_time, - **kwargs, - ) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py index 4fb97d78a919..d9a2fb1ffffb 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/streams/utils/stream_helper_async.py @@ -18,9 +18,7 @@ async def get_first_stream_slice(stream: AsyncStream) -> Optional[Mapping[str, A """ first_slice = await anext( stream.stream_slices( - cursor_field=[stream.cursor_field] - if isinstance(stream.cursor_field, str) - else stream.cursor_field, + cursor_field=stream.cursor_field, sync_mode=SyncMode.full_refresh, ) ) @@ -29,7 +27,7 @@ async def get_first_stream_slice(stream: AsyncStream) -> Optional[Mapping[str, A async def get_first_record_for_slice( stream: AsyncStream, stream_slice: Optional[Mapping[str, Any]] -) -> AsyncGenerator[StreamData, None]: +) -> StreamData: """ Gets the first record for a stream_slice of a stream. :param stream: stream @@ -37,9 +35,9 @@ async def get_first_record_for_slice( :raises StopAsyncIteration: if there is no first record to return (the read_records generator is empty) :return: StreamData containing the first record in the slice """ - # We wrap the return output of read_records() because some implementations return types that are iterable, - # but not iterators such as lists or tuples - async for record in stream.read_records( - sync_mode=SyncMode.full_refresh, stream_slice=stream_slice - ): - yield record + record = await anext( + stream.read_records( + sync_mode=SyncMode.full_refresh, stream_slice=stream_slice + ) + ) + return record diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py index 7da175c0ae94..fd1accab9674 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/availability_strategy.py @@ -4,11 +4,12 @@ import logging import typing -from typing import Dict, Optional, Tuple +from typing import Dict, Optional, Tuple, Union import requests from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy +from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.sources.streams.utils.stream_helper import ( get_first_record_for_slice, get_first_stream_slice, @@ -17,6 +18,7 @@ if typing.TYPE_CHECKING: from airbyte_cdk.sources import Source + from airbyte_cdk.sources.streams.http import HttpStream class HttpAvailabilityStrategy(AvailabilityStrategy): @@ -70,7 +72,7 @@ def _handle_http_error( stream: Stream, logger: logging.Logger, source: Optional["Source"], - error: HTTPError, + error: Union[HTTPError, HttpError], ) -> Tuple[bool, Optional[str]]: """ Override this method to define error handling for various `HTTPError`s @@ -88,7 +90,15 @@ def _handle_http_error( for some reason and the str should describe what went wrong and how to resolve the unavailability, if possible. """ - status_code = error.response.status_code + if isinstance(error, HttpError): + status_code = error.status_code + url = error.url + reason = error.reason or error.error_message + else: + # TODO: wrap synchronous codepath's errors in HttpError to delete this path + status_code = error.response.status_code + url = error.response.url + reason = error.response.reason known_status_codes = self.reasons_for_unavailable_status_codes( stream, logger, source, error ) @@ -98,7 +108,7 @@ def _handle_http_error( raise error doc_ref = self._visit_docs_message(logger, source) - reason = f"The endpoint {error.response.url} returned {status_code}: {error.response.reason}. {known_reason}. {doc_ref} " + reason = f"The endpoint {url} returned {status_code}: {reason}. {known_reason}. {doc_ref} " response_error_message = stream.parse_error_message(error) if response_error_message: reason += response_error_message diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions.py index a97884b53f8a..6a0c7f3f4fbf 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/exceptions.py @@ -3,12 +3,17 @@ # +from abc import ABC from typing import Union import requests -class BaseBackoffException(requests.exceptions.HTTPError): +class AbstractBaseBackoffException(ABC): + pass + + +class BaseBackoffException(AbstractBaseBackoffException, requests.exceptions.HTTPError): def __init__(self, request: requests.PreparedRequest, response: requests.Response, error_message: str = ""): error_message = ( error_message or f"Request URL: {request.url}, Response Code: {response.status_code}, Response Text: {response.text}" diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py index 775ad77bab55..4183a3815d0e 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http.py @@ -337,6 +337,9 @@ def _send_request( def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: return HttpError.parse_response_error_message(response) + def parse_error_message(cls, error: HttpError) -> Optional[str]: + return HttpError.parse_response_error_message(error.response) + def read_records( self, sync_mode: SyncMode, diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py index 7b749ab3356a..56cd77f3384f 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/http_base.py @@ -306,6 +306,9 @@ def get_error_display_message(self, exception: BaseException) -> Optional[str]: """ if isinstance(exception, HttpError): return self.parse_error_message(exception) + elif isinstance(exception, requests.HTTPError) and exception.response is not None: + # TODO: wrap synchronous codepath's errors in HttpError to delete this path + return self.parse_response_error_message(exception.response) return None @abstractmethod diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py index a45f9791cdbe..51dcf789783f 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/rate_limiting.py @@ -5,16 +5,23 @@ import logging import sys import time -from typing import Any, Callable, Mapping, Optional +from typing import Any, Callable, Coroutine, Mapping, Optional, Type, Union +import aiohttp import backoff -from requests import PreparedRequest, RequestException, Response, codes, exceptions +from requests import HTTPError, PreparedRequest, RequestException, Response, codes, exceptions +from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import AsyncDefaultBackoffException, AsyncUserDefinedBackoffException from airbyte_cdk.sources.streams.http.utils import HttpError -from .exceptions import DefaultBackoffException, UserDefinedBackoffException +from .exceptions import AbstractBaseBackoffException, DefaultBackoffException, UserDefinedBackoffException TRANSIENT_EXCEPTIONS = ( DefaultBackoffException, + AsyncDefaultBackoffException, + aiohttp.ClientPayloadError, + aiohttp.ServerTimeoutError, + aiohttp.ServerConnectionError, + aiohttp.ServerDisconnectedError, exceptions.ConnectTimeout, exceptions.ReadTimeout, exceptions.ConnectionError, @@ -24,7 +31,12 @@ logger = logging.getLogger("airbyte") -SendRequestCallableType = Callable[[PreparedRequest, Mapping[str, Any]], Response] +AioHttpCallableType = Callable[ + [aiohttp.ClientRequest, Mapping[str, Any]], + Coroutine[Any, Any, aiohttp.ClientResponse], +] +RequestsCallableType = Callable[[PreparedRequest, Mapping[str, Any]], Response] +SendRequestCallableType = Union[AioHttpCallableType, RequestsCallableType] def default_backoff_handler( @@ -36,27 +48,41 @@ def default_backoff_handler( def log_retry_attempt(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, HttpError): + logger.info(f"Status code: {exc.status_code}, Response Content: {exc.content}") logger.info( - f"Status code: {exc.status_code}, Response Content: {exc.content}" + f"Caught retryable error '{exc.message}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." + ) + + if isinstance(exc, RequestException): + exc = HttpError(requests_error=exc) + + logger.info( + f"Caught retryable error '{str(exc)}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." ) - logger.info( - f"Caught retryable error '{str(exc)}' after {details['tries']} tries. Waiting {details['wait']} seconds then retrying..." - ) def should_give_up(exc: Exception) -> bool: # If a non-rate-limiting related 4XX error makes it this far, it means it was unexpected and probably consistent, so we shouldn't back off - if isinstance(exc, RequestException): + if isinstance(exc, HttpError): + give_up: bool = ( + exc.status_code != codes.too_many_requests and 400 <= exc.status_code < 500 + ) + status_code = exc.status_code + + elif isinstance(exc, RequestException): + # TODO: wrap synchronous codepath's errors in HttpError to delete this path give_up: bool = ( - exc.response is not None - and exc.response.status_code != codes.too_many_requests - and 400 <= exc.response.status_code < 500 + exc.response is not None and exc.response.status_code != codes.too_many_requests and 400 <= exc.response.status_code < 500 ) - if give_up: - logger.info( - f"Giving up for returned HTTP status: {exc.response.status_code}" - ) - return give_up - # Only RequestExceptions are retryable, so if we get here, it's not retryable + status_code = exc.response if exc else None + + else: + status_code = None + give_up = True + + if give_up: + logger.info(f"Giving up for returned HTTP status: {status_code}") + + # Only RequestExceptions and HttpExceptions are retryable, so if we get here, it's not retryable return False return backoff.on_exception( @@ -72,16 +98,17 @@ def should_give_up(exc: Exception) -> bool: ) -def user_defined_backoff_handler( - max_tries: Optional[int], max_time: Optional[int] = None, **kwargs: Any +def _make_user_defined_backoff_handler( + exc_type: Type[AbstractBaseBackoffException], max_tries: Optional[int], max_time: Optional[int] = None, **kwargs: Any ) -> Callable[[SendRequestCallableType], SendRequestCallableType]: def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() - if isinstance(exc, UserDefinedBackoffException): - if exc.response: - logger.info( - f"Status code: {exc.response.status_code}, Response Content: {exc.response.content}" - ) + if isinstance(exc, exc_type): + if isinstance(exc, HttpError): + logger.info(f"Status code: {exc.status_code}, Response Content: {exc.content}") + elif exc.response: + logger.info(f"Status code: {exc.response.status_code}, Response Content: {exc.response.content}") + retry_after = exc.backoff logger.info(f"Retrying. Sleeping for {retry_after} seconds") time.sleep(retry_after + 1) # extra second to cover any fractions of second @@ -89,6 +116,8 @@ def sleep_on_ratelimit(details: Mapping[str, Any]) -> None: def log_give_up(details: Mapping[str, Any]) -> None: _, exc, _ = sys.exc_info() if isinstance(exc, RequestException): + exc = HttpError(requests_error=exc) + if isinstance(exc, (HTTPError, HttpError)): logger.error( f"Max retry limit reached. Request: {exc.request}, Response: {exc.response}" ) @@ -97,7 +126,7 @@ def log_give_up(details: Mapping[str, Any]) -> None: return backoff.on_exception( backoff.constant, - UserDefinedBackoffException, + exc_type, interval=0, # skip waiting, we'll wait in on_backoff handler on_backoff=sleep_on_ratelimit, on_giveup=log_give_up, @@ -106,3 +135,6 @@ def log_give_up(details: Mapping[str, Any]) -> None: max_time=max_time, **kwargs, ) + +user_defined_backoff_handler = lambda *args, **kwargs: _make_user_defined_backoff_handler(UserDefinedBackoffException, *args, **kwargs) +async_user_defined_backoff_handler = lambda *args, **kwargs: _make_user_defined_backoff_handler(AsyncUserDefinedBackoffException, *args, **kwargs) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py index 888e67a53e92..b9bbc5d2e2b1 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/streams/http/utils.py @@ -1,5 +1,5 @@ import json -from typing import Optional +from typing import Optional, Union import aiohttp import requests @@ -10,8 +10,9 @@ class HttpError(Exception): def __init__( self, - requests_error: Optional[requests.HTTPError] = None, + requests_error: Optional[requests.RequestException] = None, aiohttp_error: Optional[aiohttp.ClientResponseError] = None, + error_message: Optional[str] = None, ): assert ( requests_error or aiohttp_error and not (requests_error and aiohttp_error) @@ -21,6 +22,8 @@ def __init__( self._aiohttp_response_json = None self._aiohttp_response_content = None self._aiohttp_response_text = None + self._aiohttp_response = None + self.error_message = error_message @property def status_code(self) -> Optional[int]: @@ -35,7 +38,7 @@ def message(self) -> str: if self._requests_error: return str(self._requests_error) elif self._aiohttp_error: - return self._aiohttp_error.message + return self.error_message else: return "" @@ -62,6 +65,20 @@ def json(self) -> Optional[JsonType]: return self._aiohttp_response_json return "" + @property + def request(self) -> Optional[Union[requests.Request, aiohttp.RequestInfo]]: + if self._requests_error and self._requests_error.response: + return self._requests_error.request + elif self._aiohttp_error: + return self._aiohttp_error.request_info + + @property + def response(self) -> Optional[Union[requests.Response, aiohttp.ClientResponse]]: + if self._requests_error and self._requests_error.response: + return self._requests_error.response + elif self._aiohttp_error: + return self._aiohttp_response + @property def url(self) -> str: if self._requests_error and self._requests_error.request: @@ -70,6 +87,14 @@ def url(self) -> str: return str(self._aiohttp_error.request_info.url) return "" + @property + def reason(self) -> Optional[str]: + if self._requests_error and self._requests_error.request: + return self._requests_error.response.reason + elif self._aiohttp_error: + return self._aiohttp_error.message + return "" + @classmethod def parse_response_error_message(cls, response: requests.Response) -> Optional[str]: """ @@ -135,6 +160,7 @@ async def set_response_data(self, response: aiohttp.ClientResponse): response_json = None text = await response.text() # This fixed a test + self._aiohttp_response = response self._aiohttp_response_json = response_json or text self._aiohttp_response_content = await response.content.read() self._aiohttp_response_text = text diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py index 08c6e051cbb1..e62bcf5cd82b 100644 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py +++ b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/http/test_http_async.py @@ -18,10 +18,10 @@ AsyncHttpStream, AsyncHttpSubStream, ) +from airbyte_cdk.sources.streams.http.exceptions import RequestBodyException from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import ( - DefaultBackoffException, - RequestBodyException, - UserDefinedBackoffException, + AsyncDefaultBackoffException, + AsyncUserDefinedBackoffException, ) from airbyte_cdk.sources.streams.http.auth import NoAuth from airbyte_cdk.sources.streams.http.auth import ( @@ -202,7 +202,7 @@ def request_callback(*args, **kwargs): with aioresponses() as m: m.get(stream.url_base, status=429, repeat=True, callback=request_callback) - with pytest.raises(UserDefinedBackoffException): + with pytest.raises(AsyncUserDefinedBackoffException): loop.run_until_complete(read_records(stream)) assert call_counter == stream.max_retries + 1 @@ -230,7 +230,7 @@ def request_callback(*args, **kwargs): with aioresponses() as m: m.get(stream.url_base, status=429, repeat=True, callback=request_callback) - with pytest.raises(UserDefinedBackoffException) as excinfo: + with pytest.raises(AsyncUserDefinedBackoffException) as excinfo: loop.run_until_complete(read_records(stream)) assert isinstance(excinfo.value.request, aiohttp.ClientRequest) assert isinstance(excinfo.value.response, aiohttp.ClientResponse) @@ -309,7 +309,7 @@ def test_raise_on_http_errors_off_429(): with aioresponses() as m: m.get(stream.url_base, status=429, repeat=True) - with pytest.raises(DefaultBackoffException): + with pytest.raises(AsyncDefaultBackoffException): loop.run_until_complete(read_records(stream)) loop.run_until_complete(stream._session.close()) @@ -330,7 +330,7 @@ def request_callback(*args, **kwargs): m.get( stream.url_base, status=status_code, repeat=True, callback=request_callback ) - with pytest.raises(DefaultBackoffException): + with pytest.raises(AsyncDefaultBackoffException): loop.run_until_complete(read_records(stream)) assert call_counter == stream.max_retries + 1 diff --git a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py b/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py deleted file mode 100644 index bccf5d3e5ae3..000000000000 --- a/airbyte-cdk/python/unit_tests/sources/async_cdk/streams/utils/test_rate_limiting_async.py +++ /dev/null @@ -1,26 +0,0 @@ -# -# Copyright (c) 2023 Airbyte, Inc., all rights reserved. -# - -import aiohttp -import pytest -from airbyte_cdk.sources.async_cdk.streams.http.rate_limiting_async import default_backoff_handler - - -def helper_with_exceptions(exception_type): - raise exception_type - - -@pytest.mark.parametrize( - "max_tries, max_time, factor, exception_to_raise", - [ - (1, None, 1, aiohttp.ClientPayloadError), - (1, None, 1, aiohttp.ServerTimeoutError), - (2, 2, 1, aiohttp.ServerConnectionError), - (2, 2, 1, aiohttp.ServerDisconnectedError), - ], -) -def test_default_backoff_handler(max_tries: int, max_time: int, factor: int, exception_to_raise: Exception): - backoff_handler = default_backoff_handler(max_tries=max_tries, max_time=max_time, factor=factor)(helper_with_exceptions) - with pytest.raises(exception_to_raise): - backoff_handler(exception_to_raise) diff --git a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy.py b/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy.py index b63af7973854..32e96cf16717 100644 --- a/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy.py +++ b/airbyte-cdk/python/unit_tests/sources/streams/http/test_availability_strategy.py @@ -87,7 +87,7 @@ def read_records(self, *args, **kvargs): class MockResponseWithJsonContents(requests.Response, mocker.MagicMock): def __init__(self, *args, **kvargs): mocker.MagicMock.__init__(self) - requests.Response.__init__(self, **kvargs) + requests.Response.__init__(self) self.json = mocker.MagicMock() class MockSource(AbstractSource): diff --git a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting.py b/airbyte-cdk/python/unit_tests/utils/test_rate_limiting.py index d1ed294b930d..85ddf535cae6 100644 --- a/airbyte-cdk/python/unit_tests/utils/test_rate_limiting.py +++ b/airbyte-cdk/python/unit_tests/utils/test_rate_limiting.py @@ -2,6 +2,7 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # +import aiohttp import pytest from airbyte_cdk.sources.streams.http.rate_limiting import default_backoff_handler from requests import exceptions @@ -18,6 +19,10 @@ def helper_with_exceptions(exception_type): (1, 1, 0, exceptions.ReadTimeout), (2, 2, 1, exceptions.ConnectionError), (3, 3, 1, exceptions.ChunkedEncodingError), + (1, None, 1, aiohttp.ClientPayloadError), + (1, None, 1, aiohttp.ServerTimeoutError), + (2, 2, 1, aiohttp.ServerConnectionError), + (2, 2, 1, aiohttp.ServerDisconnectedError), ], ) def test_default_backoff_handler(max_tries: int, max_time: int, factor: int, exception_to_raise: Exception): diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index d323dd0a024e..3f79dc9e7624 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -87,14 +87,15 @@ async def test_bulk_sync_creation_failed(stream_config, stream_api): await stream.ensure_session() def callback(*args, **kwargs): - return CallbackResult(status=400, reason="test_error", content_type="application/json") + return CallbackResult(status=400, payload={"message": "test_error"}) with aioresponses() as m: m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) with pytest.raises(HttpError) as err: stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] - assert err.value.message == "test_error" + + assert err.value.json()["message"] == "test_error" await stream._session.close() From bfa712430e27712f96bb6b053fa5bbf0224cac8b Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Mon, 8 Jan 2024 21:13:49 -0500 Subject: [PATCH 72/74] Salesforce: start splitting into sync vs async --- .../source_salesforce/__init__.py | 4 +- .../async_salesforce/__init__.py | 3 + .../{ => async_salesforce}/api.py | 0 .../availability_strategy.py | 11 +- .../{ => async_salesforce}/exceptions.py | 0 .../{ => async_salesforce}/rate_limiting.py | 4 +- .../{ => async_salesforce}/source.py | 3 +- .../{ => async_salesforce}/streams.py | 9 +- .../{ => async_salesforce}/utils.py | 0 .../source-salesforce/unit_tests/api_test.py | 22 +- .../unit_tests/api_test_async.py | 1073 +++++++++++++++++ .../source-salesforce/unit_tests/conftest.py | 6 +- .../unit_tests/discovery_test.py | 4 +- .../unit_tests/test_memory.py | 2 +- .../unit_tests/test_memory_async.py | 53 + 15 files changed, 1160 insertions(+), 34 deletions(-) create mode 100644 airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/__init__.py rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/api.py (100%) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/availability_strategy.py (82%) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/exceptions.py (100%) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/rate_limiting.py (92%) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/source.py (99%) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/streams.py (99%) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{ => async_salesforce}/utils.py (100%) create mode 100644 airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py create mode 100644 airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py index fb3d4279e4ec..b410c64d63c2 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py @@ -2,6 +2,6 @@ # Copyright (c) 2021 Airbyte, Inc., all rights reserved. # -from .source import SourceSalesforce, SalesforceSourceDispatcher +from .async_salesforce.source import AsyncSourceSalesforce, SalesforceSourceDispatcher -__all__ = ["SourceSalesforce", "SalesforceSourceDispatcher"] +__all__ = ["AsyncSourceSalesforce", "SalesforceSourceDispatcher"] diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/__init__.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/__init__.py new file mode 100644 index 000000000000..69de4c37ac7d --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/__init__.py @@ -0,0 +1,3 @@ +from .source import AsyncSourceSalesforce, SalesforceSourceDispatcher + +__all__ = ["AsyncSourceSalesforce", "SalesforceSourceDispatcher"] diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/api.py similarity index 100% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/api.py diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/availability_strategy.py similarity index 82% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/availability_strategy.py index f8492d31588b..cbb9eac0a16f 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/availability_strategy.py @@ -8,15 +8,16 @@ from airbyte_cdk.sources.async_cdk.streams.http.availability_strategy_async import AsyncHttpAvailabilityStrategy from airbyte_cdk.sources.streams import Stream -from requests import HTTPError, codes +from airbyte_cdk.sources.streams.http.utils import HttpError +from requests import codes if typing.TYPE_CHECKING: from airbyte_cdk.sources import Source -class SalesforceAvailabilityStrategy(AsyncHttpAvailabilityStrategy): +class AsyncSalesforceAvailabilityStrategy(AsyncHttpAvailabilityStrategy): def handle_http_error( - self, stream: Stream, logger: logging.Logger, source: Optional["Source"], error: HTTPError + self, stream: Stream, logger: logging.Logger, source: Optional["Source"], error: HttpError ) -> Tuple[bool, Optional[str]]: """ There are several types of Salesforce sobjects that require additional processing: @@ -26,8 +27,8 @@ def handle_http_error( And since we use a dynamic method of generating streams for Salesforce connector - at the stage of discover, we cannot filter out these streams, so we check for them before reading from the streams. """ - if error.response.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: - error_data = error.response.json()[0] + if error.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: + error_data = error.json()[0] error_code = error_data.get("errorCode", "") if error_code != "REQUEST_LIMIT_EXCEEDED" or error_code == "INVALID_TYPE_FOR_OPERATION": return False, f"Cannot receive data for stream '{stream.name}', error message: '{error_data.get('message')}'" diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/exceptions.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/exceptions.py similarity index 100% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/exceptions.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/exceptions.py diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py similarity index 92% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py index 344a6412e024..fffb3cb9e3b1 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py @@ -6,11 +6,11 @@ import sys import backoff -from airbyte_cdk.sources.streams.http.exceptions import DefaultBackoffException +from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import AsyncDefaultBackoffException from requests import codes, exceptions # type: ignore[import] TRANSIENT_EXCEPTIONS = ( - DefaultBackoffException, + AsyncDefaultBackoffException, exceptions.ConnectTimeout, exceptions.ReadTimeout, exceptions.ConnectionError, diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py similarity index 99% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py index ec12286faabb..cd43bc7f64c3 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py @@ -6,7 +6,6 @@ from datetime import datetime from typing import Any, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union -import aiohttp import requests from airbyte_cdk import AirbyteLogger from airbyte_cdk.logger import AirbyteLogFormatter @@ -43,7 +42,7 @@ class AirbyteStopSync(AirbyteTracedException): pass -class SourceSalesforce(AsyncAbstractSource): +class AsyncSourceSalesforce(AsyncAbstractSource): DATETIME_FORMAT = "%Y-%m-%dT%H:%M:%SZ" START_DATE_OFFSET_IN_YEARS = 2 MAX_WORKERS = 5 diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py similarity index 99% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py index 56917aab5386..51cba51ab0dd 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py @@ -2,7 +2,6 @@ # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # -import asyncio import csv import ctypes import math @@ -11,8 +10,6 @@ import urllib.parse import uuid from abc import ABC -from contextlib import closing -from threading import Thread from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Type, Union import aiohttp @@ -28,11 +25,11 @@ from airbyte_cdk.utils import AirbyteTracedException from numpy import nan from pendulum import DateTime # type: ignore[attr-defined] -from requests import codes, exceptions +from requests import codes from requests.models import PreparedRequest from .api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce -from .availability_strategy import SalesforceAvailabilityStrategy +from .availability_strategy import AsyncSalesforceAvailabilityStrategy from .exceptions import SalesforceException, TmpFileIOError from .rate_limiting import default_backoff_handler @@ -91,7 +88,7 @@ def url_base(self) -> str: @property def availability_strategy(self) -> Optional["AvailabilityStrategy"]: - return SalesforceAvailabilityStrategy() + return AsyncSalesforceAvailabilityStrategy() @property def too_many_properties(self): diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/utils.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/utils.py similarity index 100% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/utils.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/utils.py diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index 3f79dc9e7624..21ccdd28402c 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -23,10 +23,10 @@ from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils import AirbyteTracedException from conftest import encoding_symbols_parameters, generate_stream -from source_salesforce.api import Salesforce -from source_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING -from source_salesforce.source import SalesforceSourceDispatcher, SourceSalesforce -from source_salesforce.streams import ( +from source_salesforce.async_salesforce.api import Salesforce +from source_salesforce.async_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING +from source_salesforce.async_salesforce.source import SalesforceSourceDispatcher, AsyncSourceSalesforce +from source_salesforce.async_salesforce.streams import ( CSV_FIELD_SIZE_LIMIT, BulkIncrementalSalesforceStream, BulkSalesforceStream, @@ -65,7 +65,7 @@ def test_login_authentication_error_handler( stream_config, requests_mock, login_status_code, login_json_resp, expected_error_msg, is_config_error ): - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) logger = logging.getLogger("airbyte") requests_mock.register_uri( "POST", "https://login.salesforce.com/services/oauth2/token", json=login_json_resp, status_code=login_status_code @@ -410,7 +410,7 @@ async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_t async def test_check_connection_rate_limit( stream_config, login_status_code, login_json_resp, discovery_status_code, discovery_resp_json, expected_error_msg ): - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) logger = logging.getLogger("airbyte") with requests_mock.Mocker() as m: @@ -449,7 +449,7 @@ def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): stream_1.page_size = 6 stream_1.state_checkpoint_interval = 5 - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = streams logger = logging.getLogger("airbyte") @@ -512,7 +512,7 @@ async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): stream_1.state_checkpoint_interval = 3 configure_request_params_mock(stream_1, stream_2) - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = [stream_1, stream_2] @@ -730,7 +730,7 @@ async def test_forwarding_sobject_options(stream_config, stream_names, catalog_s ], }, ) - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.catalog = catalog streams = source.streams(config=stream_config) expected_names = catalog_stream_names if catalog else stream_names @@ -790,7 +790,7 @@ def _get_streams(stream_config, stream_names, catalog_stream_names, sync_type) - ], }, ) - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.catalog = catalog return source.streams(config=stream_config) @@ -974,7 +974,7 @@ async def test_bulk_stream_request_params_states(stream_config_date_format, stre stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config_date_format, stream_api) await stream.ensure_session() - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = [stream] base_url = f"{stream.sf_api.instance_url}{stream.path()}" diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py new file mode 100644 index 000000000000..afd9cbb8bbb1 --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py @@ -0,0 +1,1073 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# +import asyncio +import csv +import io +import logging +import re +from datetime import datetime +from typing import List +from unittest.mock import Mock +from yarl import URL + +import freezegun +import pendulum +import pytest +import requests_mock +from aioresponses import CallbackResult, aioresponses +from airbyte_cdk.models import AirbyteStream, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, DestinationSyncMode, SyncMode, Type +from airbyte_cdk.sources.async_cdk import source_dispatcher +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade +from airbyte_cdk.sources.streams.http.utils import HttpError +from airbyte_cdk.utils import AirbyteTracedException +from conftest import encoding_symbols_parameters, generate_stream +from source_salesforce.api import Salesforce +from source_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING +from source_salesforce.async_salesforce.source import SalesforceSourceDispatcher, SourceSalesforce +from source_salesforce.streams import ( + CSV_FIELD_SIZE_LIMIT, + BulkIncrementalSalesforceStream, + BulkSalesforceStream, + BulkSalesforceSubStream, + IncrementalRestSalesforceStream, + RestSalesforceStream, +) + +_ANY_CATALOG = ConfiguredAirbyteCatalog.parse_obj({"streams": []}) +_ANY_CONFIG = {} + + +@pytest.mark.parametrize( + "login_status_code, login_json_resp, expected_error_msg, is_config_error", + [ + ( + 400, + {"error": "invalid_grant", "error_description": "expired access/refresh token"}, + AUTHENTICATION_ERROR_MESSAGE_MAPPING.get("expired access/refresh token"), + True, + ), + ( + 400, + {"error": "invalid_grant", "error_description": "Authentication failure."}, + 'An error occurred: {"error": "invalid_grant", "error_description": "Authentication failure."}', + False, + ), + ( + 401, + {"error": "Unauthorized", "error_description": "Unautorized"}, + 'An error occurred: {"error": "Unauthorized", "error_description": "Unautorized"}', + False, + ), + ], +) +def test_login_authentication_error_handler( + stream_config, requests_mock, login_status_code, login_json_resp, expected_error_msg, is_config_error +): + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + logger = logging.getLogger("airbyte") + requests_mock.register_uri( + "POST", "https://login.salesforce.com/services/oauth2/token", json=login_json_resp, status_code=login_status_code + ) + + if is_config_error: + with pytest.raises(AirbyteTracedException) as err: + source.check_connection(logger, stream_config) + assert err.value.message == expected_error_msg + else: + result, msg = source.check_connection(logger, stream_config) + assert result is False + assert msg == expected_error_msg + + +@pytest.mark.asyncio +async def test_bulk_sync_creation_failed(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + + def callback(*args, **kwargs): + return CallbackResult(status=400, payload={"message": "test_error"}) + + with aioresponses() as m: + m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) + with pytest.raises(HttpError) as err: + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + + assert err.value.json()["message"] == "test_error" + await stream._session.close() + + +@pytest.mark.asyncio +async def test_bulk_stream_fallback_to_rest(stream_config, stream_api): + """ + Here we mock BULK API with response returning error, saying BULK is not supported for this kind of entity. + On the other hand, we mock REST API for this same entity with a successful response. + After having instantiated a BulkStream, sync should succeed in case it falls back to REST API. Otherwise it would throw an error. + """ + stream = await generate_stream("CustomEntity", stream_config, stream_api) + await stream.ensure_session() + + def callback(*args, **kwargs): + return CallbackResult(status=400, payload={"errorCode": "INVALIDENTITY", "message": "CustomEntity is not supported by the Bulk API"}, content_type="application/json") + + rest_stream_records = [ + {"id": 1, "name": "custom entity", "created": "2010-11-11"}, + {"id": 11, "name": "custom entity", "created": "2020-01-02"}, + ] + async def get_records(*args, **kwargs): + nonlocal rest_stream_records + for record in rest_stream_records: + yield record + + with aioresponses() as m: + # mock a BULK API + m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) + # mock REST API + stream.read_records = get_records + assert type(stream) is BulkIncrementalSalesforceStream + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + assert [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] == rest_stream_records + + await stream._session.close() + + +@pytest.mark.asyncio +async def test_stream_unsupported_by_bulk(stream_config, stream_api): + """ + Stream `AcceptedEventRelation` is not supported by BULK API, so that REST API stream will be used for it. + """ + stream_name = "AcceptedEventRelation" + stream = await generate_stream(stream_name, stream_config, stream_api) + assert not isinstance(stream, BulkSalesforceStream) + + +@pytest.mark.asyncio +async def test_stream_contains_unsupported_properties_by_bulk(stream_config, stream_api_v2): + """ + Stream `Account` contains compound field such as BillingAddress, which is not supported by BULK API (csv), + in that case REST API stream will be used for it. + """ + stream_name = "Account" + stream = await generate_stream(stream_name, stream_config, stream_api_v2) + assert not isinstance(stream, BulkSalesforceStream) + + +@pytest.mark.asyncio +async def test_bulk_sync_pagination(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + job_id = "fake_job" + call_counter = 0 + + def cb1(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + return CallbackResult(headers={"Sforce-Locator": "somelocator_1"}, body="\n".join(resp_text)) + + def cb2(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + return CallbackResult(headers={"Sforce-Locator": "somelocator_2"}, body="\n".join(resp_text)) + + def cb3(*args, **kwargs): + nonlocal call_counter + call_counter += 1 + return CallbackResult(headers={"Sforce-Locator": "null"}, body="\n".join(resp_text)) + + with aioresponses() as m: + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + m.post(f"{base_url}", callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.get(f"{base_url}/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + resp_text = ["Field1,LastModifiedDate,ID"] + [f"test,2021-11-16,{i}" for i in range(5)] + m.get(f"{base_url}/{job_id}/results", callback=cb1) + m.get(f"{base_url}/{job_id}/results?locator=somelocator_1", callback=cb2) + m.get(f"{base_url}/{job_id}/results?locator=somelocator_2", callback=cb3) + m.delete(base_url + f"/{job_id}") + + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + loaded_ids = [int(record["ID"]) async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + assert loaded_ids == [0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4] + assert call_counter == 3 + await stream._session.close() + + + + +def _prepare_mock(m, stream): + job_id = "fake_job_1" + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.delete(base_url + f"/{job_id}") + m.get(base_url + f"/{job_id}/results", callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2021-11-16,1")) + m.patch(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(body="")) + return job_id + + +async def _get_result_id(stream): + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + records = [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + return int(list(records)[0]["ID"]) + + +@pytest.mark.asyncio +async def test_bulk_sync_successful(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + + with aioresponses() as m: + job_id = _prepare_mock(m, stream) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + assert await _get_result_id(stream) == 1 + + +@pytest.mark.asyncio +async def test_bulk_sync_successful_long_response(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + job_id = _prepare_mock(m, stream) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "UploadComplete", "id": job_id})) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete", "id": job_id})) + assert await _get_result_id(stream) == 1 + + +# maximum timeout is wait_timeout * max_retry_attempt +# this test tries to check a job state 17 times with +-1second for very one +@pytest.mark.asyncio +@pytest.mark.timeout(17) +async def test_bulk_sync_successful_retry(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + job_id = _prepare_mock(m, stream) + # 2 failed attempts, 3rd one should be successful + states = [{"json": {"state": "InProgress", "id": job_id}}] * 17 + states.append({"json": {"state": "JobComplete", "id": job_id}}) + # raise Exception(states) + for _ in range(17): + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) + m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete", "id": job_id})) + + assert await _get_result_id(stream) == 1 + + +@pytest.mark.asyncio +@pytest.mark.timeout(30) +async def test_bulk_sync_failed_retry(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds + await stream.ensure_session() + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + job_id = _prepare_mock(m, stream) + m.get(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) + m.post(base_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + with pytest.raises(Exception) as err: + stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) + [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + assert "stream using BULK API was failed" in str(err.value) + + await stream._session.close() + + +@pytest.mark.asyncio +@pytest.mark.parametrize( + "start_date_provided,stream_name,expected_start_date", + [ + (True, "Account", "2010-01-18T21:18:20Z"), + (True, "ActiveFeatureLicenseMetric", "2010-01-18T21:18:20Z"), + ], +) +async def test_stream_start_date( + start_date_provided, + stream_name, + expected_start_date, + stream_config, + stream_api, + stream_config_without_start_date, +): + if start_date_provided: + stream = await generate_stream(stream_name, stream_config, stream_api) + assert stream.start_date == expected_start_date + else: + stream = await generate_stream(stream_name, stream_config_without_start_date, stream_api) + assert datetime.strptime(stream.start_date, "%Y-%m-%dT%H:%M:%SZ").year == datetime.now().year - 2 + + +@pytest.mark.asyncio +async def test_stream_start_date_should_be_converted_to_datetime_format(stream_config_date_format, stream_api): + stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) + assert stream.start_date == "2010-01-18T00:00:00Z" + + +@pytest.mark.asyncio +async def test_stream_start_datetime_format_should_not_changed(stream_config, stream_api): + stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config, stream_api) + assert stream.start_date == "2010-01-18T21:18:20Z" + + +@pytest.mark.asyncio +async def test_download_data_filter_null_bytes(stream_config, stream_api): + job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + + with aioresponses() as m: + m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b"\x00")) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + res = list(stream.read_with_chunks(tmp_file, response_encoding)) + assert res == [] + + m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b'"Id","IsDeleted"\n\x00"0014W000027f6UwQAI","false"\n\x00\x00')) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + res = list(stream.read_with_chunks(tmp_file, response_encoding)) + assert res == [{"Id": "0014W000027f6UwQAI", "IsDeleted": "false"}] + + +@pytest.mark.asyncio +async def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): + job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + + with aioresponses() as m: + m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b'"IsDeleted","Age"\n"false",24\n')) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + res = list(stream.read_with_chunks(tmp_file, response_encoding)) + assert res == [{"IsDeleted": "false", "Age": "24"}] + + +@pytest.mark.asyncio +async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): + job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + + with aioresponses() as m: + m.get(job_full_url_results, body=b'"ZipCode"\n"01234"\n') + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + res = list(stream.read_with_chunks(tmp_file, response_encoding)) + assert res == [{"ZipCode": "01234"}] + + +@pytest.mark.asyncio +async def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): + job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + + with aioresponses() as m: + m.get(job_full_url_results, body=b'"IsDeleted","Age","Name"\n"false",,"Airbyte"\n') + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + res = list(stream.read_with_chunks(tmp_file, response_encoding)) + assert res == [{"IsDeleted": "false", "Age": None, "Name": "Airbyte"}] + + +@pytest.mark.asyncio +@pytest.mark.parametrize( + "chunk_size, content_type_header, content, expected_result", + encoding_symbols_parameters(), + ids=[f"charset: {x[1]}, chunk_size: {x[0]}" for x in encoding_symbols_parameters()], +) +async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): + job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + + with aioresponses() as m: + m.get(job_full_url_results, headers=content_type_header, body=content) + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + res = list(stream.read_with_chunks(tmp_file, response_encoding)) + assert res == expected_result + + +@pytest.mark.parametrize( + "login_status_code, login_json_resp, discovery_status_code, discovery_resp_json, expected_error_msg", + ( + (403, [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."}], 200, {}, "API Call limit is exceeded"), + ( + 200, + {"access_token": "access_token", "instance_url": "https://instance_url"}, + 403, + [{"errorCode": "FORBIDDEN", "message": "You do not have enough permissions"}], + 'An error occurred: [{"errorCode": "FORBIDDEN", "message": "You do not have enough permissions"}]', + ), + ), +) +async def test_check_connection_rate_limit( + stream_config, login_status_code, login_json_resp, discovery_status_code, discovery_resp_json, expected_error_msg +): + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + logger = logging.getLogger("airbyte") + + with requests_mock.Mocker() as m: + m.register_uri("POST", "https://login.salesforce.com/services/oauth2/token", json=login_json_resp, status_code=login_status_code) + m.register_uri( + "GET", "https://instance_url/services/data/v57.0/sobjects", json=discovery_resp_json, status_code=discovery_status_code + ) + result, msg = source.check_connection(logger, stream_config) + assert result is False + assert msg == expected_error_msg + + +def configure_request_params_mock(stream_1, stream_2): + stream_1.request_params = Mock() + stream_1.request_params.return_value = {"q": "query"} + + stream_2.request_params = Mock() + stream_2.request_params.return_value = {"q": "query"} + + +def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): + """ + Connector should stop the sync if one stream reached rate limit + stream_1, stream_2, stream_3, ... + While reading `stream_1` if 403 (Rate Limit) is received, it should finish that stream with success and stop the sync process. + Next streams should not be executed. + """ + source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time + stream_config.update({"start_date": "2021-10-01"}) + loop = asyncio.get_event_loop() + stream_1: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Account", stream_config, stream_api)) + stream_2: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Asset", stream_config, stream_api)) + streams = [stream_1, stream_2] + configure_request_params_mock(stream_1, stream_2) + + stream_1.page_size = 6 + stream_1.state_checkpoint_interval = 5 + + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source.streams = Mock() + source.streams.return_value = streams + logger = logging.getLogger("airbyte") + + json_response = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} + + orig_read_stream = source.async_source.read_stream + + async def patched_read_stream(*args, **kwargs): + base_url = f"{stream_1.sf_api.instance_url}{stream_1.path()}" + with aioresponses() as m: + creation_responses = [] + for page in [1, 2]: + job_id = f"fake_job_{page}_{stream_1.name}" + creation_responses.append({"id": job_id}) + + m.get(base_url + f"/{job_id}", callback=lambda *_, **__: CallbackResult(payload={"state": "JobComplete"})) + + resp = ["Field1,LastModifiedDate,Id"] + [f"test,2021-10-0{i},{i}" for i in range(1, 7)] # 6 records per page + + if page == 1: + # Read the first page successfully + m.get(base_url + f"/{job_id}/results", callback=lambda *_, **__: CallbackResult(body="\n".join(resp))) + else: + # Requesting for results when reading second page should fail with 403 (Rate Limit error) + m.get(base_url + f"/{job_id}/results", status=403, callback=lambda *_, **__: CallbackResult(status=403, payload=json_response)) + + m.delete(base_url + f"/{job_id}") + + def cb(response): + return lambda *_, **__: CallbackResult(payload=response) + + for response in creation_responses: + m.post(base_url, callback=cb(response)) + + async for r in orig_read_stream(**kwargs): + yield r + + source.async_source.read_stream = patched_read_stream + + result = [i for i in source.read(logger=logger, config=stream_config, catalog=bulk_catalog, state=state)] + assert stream_1.request_params.called + assert ( + not stream_2.request_params.called + ), "The second stream should not be executed, because the first stream finished with Rate Limit." + + records = [item for item in result if item.type == Type.RECORD] + assert len(records) == 6 # stream page size: 6 + + state_record = [item for item in result if item.type == Type.STATE][0] + assert state_record.state.data["Account"]["LastModifiedDate"] == "2021-10-05T00:00:00+00:00" # state checkpoint interval is 5. + + +@pytest.mark.asyncio +async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): + source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time + stream_config.update({"start_date": "2021-11-01"}) + stream_1: IncrementalRestSalesforceStream = await generate_stream("KnowledgeArticle", stream_config, stream_api) + stream_2: IncrementalRestSalesforceStream = await generate_stream("AcceptedEventRelation", stream_config, stream_api) + stream_1.state_checkpoint_interval = 3 + configure_request_params_mock(stream_1, stream_2) + + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source.streams = Mock() + source.streams.return_value = [stream_1, stream_2] + + logger = logging.getLogger("airbyte") + + next_page_url = "/services/data/v57.0/query/012345" + response_1 = { + "done": False, + "totalSize": 10, + "nextRecordsUrl": next_page_url, + "records": [ + { + "ID": 1, + "LastModifiedDate": "2021-11-15", + }, + { + "ID": 2, + "LastModifiedDate": "2021-11-16", + }, + { + "ID": 3, + "LastModifiedDate": "2021-11-17", # check point interval + }, + { + "ID": 4, + "LastModifiedDate": "2021-11-18", + }, + { + "ID": 5, + "LastModifiedDate": "2021-11-19", + }, + ], + } + response_2 = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} + + def cb1(*args, **kwargs): + return CallbackResult(payload=response_1, status=200) + + def cb2(*args, **kwargs): + return CallbackResult(payload=response_2, status=403, reason="") + + orig_read_records_s1 = stream_1.read_records + orig_read_records_s2 = stream_2.read_records + + async def patched_read_records_s1(*args, **kwargs): + with aioresponses() as m: + m.get(re.compile(re.escape(rf"{stream_1.sf_api.instance_url}{stream_1.path()}") + rf"\??.*"), repeat=True, callback=cb1) + m.get(re.compile(re.escape(rf"{stream_1.sf_api.instance_url}{next_page_url}") + rf"\??.*"), repeat=True, callback=cb2) + + async for r in orig_read_records_s1(**kwargs): + yield r + + async def patched_read_records_s2(*args, **kwargs): + with aioresponses() as m: + m.get(re.compile(re.escape(rf"{stream_2.sf_api.instance_url}{stream_2.path()}") + rf"\??.*"), repeat=True, callback=cb1) + m.get(re.compile(re.escape(rf"{stream_2.sf_api.instance_url}{next_page_url}") + rf"\??.*"), repeat=True, callback=cb1) + async for r in orig_read_records_s2(**kwargs): + yield r + + async def check_availability(*args, **kwargs): + return (True, None) + + stream_1.read_records = lambda *args, **kwargs: patched_read_records_s1(stream_1, *args, **kwargs) + stream_1.check_availability = check_availability + stream_2.read_records = lambda *args, **kwargs: patched_read_records_s2(stream_2, *args, **kwargs) + stream_2.check_availability = check_availability + + result = [i for i in source.read(logger=logger, config=stream_config, catalog=rest_catalog, state=state)] + + assert stream_1.request_params.called + assert ( + not stream_2.request_params.called + ), "The second stream should not be executed, because the first stream finished with Rate Limit." + + records = [item for item in result if item.type == Type.RECORD] + assert len(records) == 5 + + state_record = [item for item in result if item.type == Type.STATE][0] + assert state_record.state.data["KnowledgeArticle"]["LastModifiedDate"] == "2021-11-17T00:00:00+00:00" + + +@pytest.mark.asyncio +async def test_pagination_rest(stream_config, stream_api): + stream_name = "AcceptedEventRelation" + stream: RestSalesforceStream = await generate_stream(stream_name, stream_config, stream_api) + stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds + next_page_url = "/services/data/v57.0/query/012345" + await stream.ensure_session() + + resp_1 = { + "done": False, + "totalSize": 4, + "nextRecordsUrl": next_page_url, + "records": [ + { + "ID": 1, + "LastModifiedDate": "2021-11-15", + }, + { + "ID": 2, + "LastModifiedDate": "2021-11-16", + }, + ], + } + resp_2 = { + "done": True, + "totalSize": 4, + "records": [ + { + "ID": 3, + "LastModifiedDate": "2021-11-17", + }, + { + "ID": 4, + "LastModifiedDate": "2021-11-18", + }, + ], + } + + with aioresponses() as m: + m.get(re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0\??.*"), callback=lambda *args, **kwargs: CallbackResult(payload=resp_1)) + m.get("https://fase-account.salesforce.com" + next_page_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload=resp_2)) + + records = [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh)] + assert len(records) == 4 + + +@pytest.mark.asyncio +async def test_csv_reader_dialect_unix(): + stream: BulkSalesforceStream = BulkSalesforceStream(stream_name=None, sf_api=None, pk=None) + url_results = "https://fake-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + await stream.ensure_session() + + data = [ + {"Id": "1", "Name": '"first_name" "last_name"'}, + {"Id": "2", "Name": "'" + 'first_name"\n' + "'" + 'last_name\n"'}, + {"Id": "3", "Name": "first_name last_name"}, + ] + + with io.StringIO("", newline="") as csvfile: + writer = csv.DictWriter(csvfile, fieldnames=["Id", "Name"], dialect="unix") + writer.writeheader() + for line in data: + writer.writerow(line) + text = csvfile.getvalue() + + with aioresponses() as m: + m.get(url_results, callback=lambda *args, **kwargs: CallbackResult(body=text)) + tmp_file, response_encoding, _ = await stream.download_data(url=url_results) + result = [i for i in stream.read_with_chunks(tmp_file, response_encoding)] + assert result == data + + +@pytest.mark.parametrize( + "stream_names,catalog_stream_names,", + ( + ( + ["stream_1", "stream_2", "Describe"], + None, + ), + ( + ["stream_1", "stream_2"], + ["stream_1", "stream_2", "Describe"], + ), + ( + ["stream_1", "stream_2", "stream_3", "Describe"], + ["stream_1", "Describe"], + ), + ), +) +async def test_forwarding_sobject_options(stream_config, stream_names, catalog_stream_names) -> None: + sobjects_matcher = re.compile("/sobjects$") + token_matcher = re.compile("/token$") + describe_matcher = re.compile("/describe$") + catalog = None + if catalog_stream_names: + catalog = ConfiguredAirbyteCatalog( + streams=[ + ConfiguredAirbyteStream( + stream=AirbyteStream( + name=catalog_stream_name, supported_sync_modes=[SyncMode.full_refresh], json_schema={"type": "object"} + ), + sync_mode=SyncMode.full_refresh, + destination_sync_mode=DestinationSyncMode.overwrite, + ) + for catalog_stream_name in catalog_stream_names + ] + ) + with requests_mock.Mocker() as m: + m.register_uri("POST", token_matcher, json={"instance_url": "https://fake-url.com", "access_token": "fake-token"}) + m.register_uri( + "GET", + describe_matcher, + json={ + "fields": [ + { + "name": "field", + "type": "string", + } + ] + }, + ) + m.register_uri( + "GET", + sobjects_matcher, + json={ + "sobjects": [ + { + "name": stream_name, + "flag1": True, + "queryable": True, + } + for stream_name in stream_names + if stream_name != "Describe" + ], + }, + ) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source.catalog = catalog + streams = source.streams(config=stream_config) + expected_names = catalog_stream_names if catalog else stream_names + assert not set(expected_names).symmetric_difference(set(stream.name for stream in streams)), "doesn't match excepted streams" + + for stream in streams: + if stream.name != "Describe": + if isinstance(stream, StreamFacade): + assert stream._legacy_stream.sobject_options == {"flag1": True, "queryable": True} + else: + assert stream.sobject_options == {"flag1": True, "queryable": True} + return + + +def _get_streams(stream_config, stream_names, catalog_stream_names, sync_type) -> List[Stream]: + sobjects_matcher = re.compile("/sobjects$") + token_matcher = re.compile("/token$") + describe_matcher = re.compile("/describe$") + catalog = None + if catalog_stream_names: + catalog = ConfiguredAirbyteCatalog( + streams=[ + ConfiguredAirbyteStream( + stream=AirbyteStream(name=catalog_stream_name, supported_sync_modes=[sync_type], json_schema={"type": "object"}), + sync_mode=sync_type, + destination_sync_mode=DestinationSyncMode.overwrite, + ) + for catalog_stream_name in catalog_stream_names + ] + ) + with requests_mock.Mocker() as m: + m.register_uri("POST", token_matcher, json={"instance_url": "https://fake-url.com", "access_token": "fake-token"}) + m.register_uri( + "GET", + describe_matcher, + json={ + "fields": [ + { + "name": "field", + "type": "string", + } + ] + }, + ) + m.register_uri( + "GET", + sobjects_matcher, + json={ + "sobjects": [ + { + "name": stream_name, + "flag1": True, + "queryable": True, + } + for stream_name in stream_names + if stream_name != "Describe" + ], + }, + ) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source.catalog = catalog + return source.streams(config=stream_config) + + +def test_csv_field_size_limit(): + DEFAULT_CSV_FIELD_SIZE_LIMIT = 1024 * 128 + + field_size = 1024 * 1024 + text = '"Id","Name"\n"1","' + field_size * "a" + '"\n' + + csv.field_size_limit(DEFAULT_CSV_FIELD_SIZE_LIMIT) + reader = csv.reader(io.StringIO(text)) + with pytest.raises(csv.Error): + for _ in reader: + pass + + csv.field_size_limit(CSV_FIELD_SIZE_LIMIT) + reader = csv.reader(io.StringIO(text)) + for _ in reader: + pass + + +@pytest.mark.asyncio +async def test_convert_to_standard_instance(stream_config, stream_api): + bulk_stream = await generate_stream("Account", stream_config, stream_api) + rest_stream = bulk_stream.get_standard_instance() + assert isinstance(rest_stream, IncrementalRestSalesforceStream) + + +@pytest.mark.asyncio +async def test_rest_stream_init_with_too_many_properties(stream_config, stream_api_v2_too_many_properties): + with pytest.raises(AssertionError): + # v2 means the stream is going to be a REST stream. + # A missing primary key is not allowed + await generate_stream("Account", stream_config, stream_api_v2_too_many_properties) + + +@pytest.mark.asyncio +async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): + stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + await stream.ensure_session() + chunks = list(stream.chunk_properties()) + for chunk in chunks: + assert stream.primary_key in chunk + chunks_len = len(chunks) + assert stream.too_many_properties + assert stream.primary_key + assert type(stream) == RestSalesforceStream + next_page_url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" + url_pattern = re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0/queryAll\??.*") + with aioresponses() as m: + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={ + "records": [ + {"Id": 1, "propertyA": "A"}, + {"Id": 2, "propertyA": "A"}, + {"Id": 3, "propertyA": "A"}, + {"Id": 4, "propertyA": "A"}, + ] + })) + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"nextRecordsUrl": next_page_url, "records": [{"Id": 1, "propertyB": "B"}, {"Id": 2, "propertyB": "B"}]})) + # 2 for 2 chunks above + for _ in range(chunks_len - 2): + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 1}, {"Id": 2}], "nextRecordsUrl": next_page_url})) + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 3, "propertyB": "B"}, {"Id": 4, "propertyB": "B"}]})) + # 2 for 1 chunk above and 1 chunk had no next page + for _ in range(chunks_len - 2): + m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 3}, {"Id": 4}]})) + + records = [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh)] + assert records == [ + {"Id": 1, "propertyA": "A", "propertyB": "B"}, + {"Id": 2, "propertyA": "A", "propertyB": "B"}, + {"Id": 3, "propertyA": "A", "propertyB": "B"}, + {"Id": 4, "propertyA": "A", "propertyB": "B"}, + ] + for call in requests_mock.request_history: + assert len(call.url) < Salesforce.REQUEST_SIZE_LIMITS + + +@pytest.mark.asyncio +async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties): + stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + chunks = list(stream.chunk_properties()) + for chunk in chunks: + assert stream.primary_key in chunk + assert stream.too_many_properties + assert stream.primary_key + assert type(stream) == RestSalesforceStream + url = re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0/queryAll\??.*") + await stream.ensure_session() + + with aioresponses() as m: + m.get(url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"records": []})) + records = [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh)] + assert records == [] + + +@pytest.mark.asyncio +@pytest.mark.parametrize( + "status_code,response_json,log_message", + [ + ( + 400, + {"errorCode": "INVALIDENTITY", "message": "Account is not supported by the Bulk API"}, + "Account is not supported by the Bulk API", + ), + (403, {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "API limit reached"}, "API limit reached"), + (400, {"errorCode": "API_ERROR", "message": "API does not support query"}, "The stream 'Account' is not queryable,"), + ( + 400, + {"errorCode": "LIMIT_EXCEEDED", "message": "Max bulk v2 query jobs (10000) per 24 hrs has been reached (10021)"}, + "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed.", + ), + ], +) +async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api, status_code, response_json, log_message, caplog): + stream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query" + + with aioresponses() as m: + m.post(url, status=status_code, callback=lambda *args, **kwargs: CallbackResult(status=status_code, payload=response_json, reason="")) + query = "Select Id, Subject from Account" + with caplog.at_level(logging.ERROR): + assert await stream.create_stream_job(query, url) is None, "this stream should be skipped" + + # check logs + assert log_message in caplog.records[-1].message + + +@pytest.mark.asyncio +@pytest.mark.parametrize( + "status_code,response_json,error_message", + [ + ( + 400, + { + "errorCode": "TXN_SECURITY_METERING_ERROR", + "message": "We can't complete the action because enabled transaction security policies took too long to complete.", + }, + 'A transient authentication error occurred. To prevent future syncs from failing, assign the "Exempt from Transaction Security" user permission to the authenticated user.', + ), + ], +) +async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, status_code, response_json, error_message): + stream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query/queryJobId" + + with aioresponses() as m: + m.get(url, status=status_code, callback=lambda *args, **kwargs: CallbackResult(status=status_code, payload=response_json, reason="")) + with pytest.raises(AirbyteTracedException) as e: + await stream.wait_for_job(url=url) + assert e.value.message == error_message + + + +@pytest.mark.asyncio() +@freezegun.freeze_time("2023-01-01") +async def test_bulk_stream_slices(stream_config_date_format, stream_api): + stream: BulkIncrementalSalesforceStream = await generate_stream("FakeBulkStream", stream_config_date_format, stream_api) + stream_slices = [s async for s in stream.stream_slices(sync_mode=SyncMode.full_refresh)] + expected_slices = [] + today = pendulum.today(tz="UTC") + start_date = pendulum.parse(stream.start_date, tz="UTC") + while start_date < today: + expected_slices.append( + { + "start_date": start_date.isoformat(timespec="milliseconds"), + "end_date": min(today, start_date.add(days=stream.STREAM_SLICE_STEP)).isoformat(timespec="milliseconds"), + } + ) + start_date = start_date.add(days=stream.STREAM_SLICE_STEP) + assert expected_slices == stream_slices + + +@pytest.mark.asyncio +@freezegun.freeze_time("2023-04-01") +async def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog): + stream_config_date_format.update({"start_date": "2023-01-01"}) + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config_date_format, stream_api) + await stream.ensure_session() + + source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source.streams = Mock() + source.streams.return_value = [stream] + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + job_id_1 = "fake_job_1" + job_id_2 = "fake_job_2" + job_id_3 = "fake_job_3" + + with aioresponses() as m: + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_1})) + m.get(base_url + f"/{job_id_1}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.delete(base_url + f"/{job_id_1}") + m.get(base_url + f"/{job_id_1}/results", + callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-01-15,1")) + m.patch(base_url + f"/{job_id_1}") + + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_2})) + m.get(base_url + f"/{job_id_2}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.delete(base_url + f"/{job_id_2}") + m.get(base_url + f"/{job_id_2}/results", + callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-04-01,2\ntest,2023-02-20,22")) + m.patch(base_url + f"/{job_id_2}") + + m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_3})) + m.get(base_url + f"/{job_id_3}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.delete(base_url + f"/{job_id_3}") + m.get(base_url + f"/{job_id_3}/results", + callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-04-01,3")) + m.patch(base_url + f"/{job_id_3}") + + logger = logging.getLogger("airbyte") + state = {"Account": {"LastModifiedDate": "2023-01-01T10:10:10.000Z"}} + bulk_catalog.streams.pop(1) + result = [i for i in source.read(logger=logger, config=stream_config_date_format, catalog=bulk_catalog, state=state)] + + actual_state_values = [item.state.data.get("Account").get(stream.cursor_field) for item in result if item.type == Type.STATE] + queries_history = m.requests + + # assert request params + assert ( + "LastModifiedDate >= 2023-01-01T10:10:10.000+00:00 AND LastModifiedDate < 2023-01-31T10:10:10.000+00:00" + in queries_history[("POST", URL(base_url))][0].kwargs["json"]["query"] + ) + assert ( + "LastModifiedDate >= 2023-01-31T10:10:10.000+00:00 AND LastModifiedDate < 2023-03-02T10:10:10.000+00:00" + in queries_history[("POST", URL(base_url))][1].kwargs["json"]["query"] + ) + assert ( + "LastModifiedDate >= 2023-03-02T10:10:10.000+00:00 AND LastModifiedDate < 2023-04-01T00:00:00.000+00:00" + in queries_history[("POST", URL(base_url))][2].kwargs["json"]["query"] + ) + + # assert states + expected_state_values = ["2023-01-15T00:00:00+00:00", "2023-03-02T10:10:10+00:00", "2023-04-01T00:00:00+00:00"] + assert actual_state_values == expected_state_values + + +@pytest.mark.asyncio +async def test_request_params_incremental(stream_config_date_format, stream_api): + stream = await generate_stream("ContentDocument", stream_config_date_format, stream_api) + params = stream.request_params(stream_state={}, stream_slice={'start_date': '2020', 'end_date': '2021'}) + + assert params == {'q': 'SELECT LastModifiedDate, Id FROM ContentDocument WHERE LastModifiedDate >= 2020 AND LastModifiedDate < 2021'} + + +@pytest.mark.asyncio +async def test_request_params_substream(stream_config_date_format, stream_api): + stream = await generate_stream("ContentDocumentLink", stream_config_date_format, stream_api) + params = stream.request_params(stream_state={}, stream_slice={'parents': [{'Id': 1}, {'Id': 2}]}) + + assert params == {"q": "SELECT LastModifiedDate, Id FROM ContentDocumentLink WHERE ContentDocumentId IN ('1','2')"} + + +@pytest.mark.asyncio +@freezegun.freeze_time("2023-03-20") +async def test_stream_slices_for_substream(stream_config, stream_api): + stream_config['start_date'] = '2023-01-01' + stream: BulkSalesforceSubStream = await generate_stream("ContentDocumentLink", stream_config, stream_api) + stream.SLICE_BATCH_SIZE = 2 # each ContentDocumentLink should contain 2 records from parent ContentDocument stream + await stream.ensure_session() + + job_id = "fake_job" + base_url = f"{stream.sf_api.instance_url}{stream.path()}" + + with aioresponses() as m: + m.post(base_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.get(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) + m.get(base_url + f"/{job_id}/results", repeat=True, callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2021-11-16,123", headers={"Sforce-Locator": "null"})) + m.delete(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult()) + + stream_slices = [slice async for slice in stream.stream_slices(sync_mode=SyncMode.full_refresh)] + assert stream_slices == [ + {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}, + {'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]}, + {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]} + ] diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py index b855715b5a1f..cb0689e24061 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py @@ -7,8 +7,8 @@ import pytest from airbyte_cdk.models import ConfiguredAirbyteCatalog -from source_salesforce.api import Salesforce -from source_salesforce.source import SourceSalesforce +from source_salesforce.async_salesforce.api import Salesforce +from source_salesforce.async_salesforce.source import AsyncSourceSalesforce @pytest.fixture(autouse=True) @@ -120,7 +120,7 @@ def stream_api_v2_pk_too_many_properties(stream_config): async def generate_stream(stream_name, stream_config, stream_api): - return (await SourceSalesforce.generate_streams(stream_config, {stream_name: None}, stream_api))[0] + return (await AsyncSourceSalesforce.generate_streams(stream_config, {stream_name: None}, stream_api))[0] def encoding_symbols_parameters(): diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py index f0151e97b036..42dfdcb047be 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py @@ -5,8 +5,8 @@ from unittest.mock import Mock import pytest -from source_salesforce.api import DATE_TYPES, LOOSE_TYPES, NUMBER_TYPES, STRING_TYPES, Salesforce -from source_salesforce.exceptions import TypeSalesforceException +from source_salesforce.async_salesforce.api import DATE_TYPES, LOOSE_TYPES, NUMBER_TYPES, STRING_TYPES, Salesforce +from source_salesforce.async_salesforce.exceptions import TypeSalesforceException @pytest.mark.parametrize( diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py index bbf2e230127a..510c19494e75 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py @@ -8,7 +8,7 @@ import pytest from aioresponses import CallbackResult, aioresponses from conftest import generate_stream -from source_salesforce.streams import BulkIncrementalSalesforceStream +from source_salesforce.async_salesforce.streams import BulkIncrementalSalesforceStream @pytest.mark.parametrize( diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py new file mode 100644 index 000000000000..510c19494e75 --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py @@ -0,0 +1,53 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + + +import tracemalloc + +import pytest +from aioresponses import CallbackResult, aioresponses +from conftest import generate_stream +from source_salesforce.async_salesforce.streams import BulkIncrementalSalesforceStream + + +@pytest.mark.parametrize( + "n_records, first_size, first_peak", + ( + (1000, 0.4, 1), + (10000, 1, 2), + (100000, 4, 9), + (200000, 7, 19), + ), + ids=[ + "1k recods", + "10k records", + "100k records", + "200k records", + ], +) +@pytest.mark.asyncio +async def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): + job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" + stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + await stream.ensure_session() + content = b'"Id","IsDeleted"' + for _ in range(n_records): + content += b'"0014W000027f6UwQAI","false"\n' + + def callback(url, **kwargs): + return CallbackResult(body=content) + + with aioresponses() as m: + m.get(job_full_url_results, status=200, callback=callback) + tracemalloc.start() + tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + for x in stream.read_with_chunks(tmp_file, response_encoding): + pass + fs, fp = tracemalloc.get_traced_memory() + first_size_in_mb, first_peak_in_mb = fs / 1024**2, fp / 1024**2 + + assert first_size_in_mb < first_size + assert first_peak_in_mb < first_peak + + await stream._session.close() From 6e93ca91fd377fa807d346016f17530e84d0e42e Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Mon, 8 Jan 2024 21:56:15 -0500 Subject: [PATCH 73/74] Split out Salesforce async --- .../connectors/source-salesforce/main.py | 8 +- .../{async_salesforce => }/api.py | 0 .../async_salesforce/rate_limiting.py | 17 +- .../async_salesforce/source.py | 2 +- .../async_salesforce/streams.py | 8 +- .../availability_strategy.py | 35 + .../{async_salesforce => }/exceptions.py | 0 .../source_salesforce/rate_limiting.py | 51 ++ .../source_salesforce/source.py | 243 +++++ .../source_salesforce/streams.py | 790 ++++++++++++++++ .../{async_salesforce => }/utils.py | 0 .../source-salesforce/unit_tests/api_test.py | 857 ++++++++---------- .../unit_tests/api_test_async.py | 88 +- .../source-salesforce/unit_tests/conftest.py | 9 +- .../unit_tests/discovery_test.py | 4 +- .../unit_tests/test_memory.py | 21 +- .../unit_tests/test_memory_async.py | 4 +- 17 files changed, 1593 insertions(+), 544 deletions(-) rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{async_salesforce => }/api.py (100%) create mode 100644 airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{async_salesforce => }/exceptions.py (100%) create mode 100644 airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py create mode 100644 airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py create mode 100644 airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py rename airbyte-integrations/connectors/source-salesforce/source_salesforce/{async_salesforce => }/utils.py (100%) diff --git a/airbyte-integrations/connectors/source-salesforce/main.py b/airbyte-integrations/connectors/source-salesforce/main.py index 7643155bde89..f2f93f940e73 100644 --- a/airbyte-integrations/connectors/source-salesforce/main.py +++ b/airbyte-integrations/connectors/source-salesforce/main.py @@ -10,7 +10,7 @@ from airbyte_cdk.entrypoint import AirbyteEntrypoint, launch from airbyte_cdk.models import AirbyteErrorTraceMessage, AirbyteMessage, AirbyteTraceMessage, TraceType, Type -from source_salesforce import SalesforceSourceDispatcher, SourceSalesforce +from source_salesforce import SalesforceSourceDispatcher, AsyncSourceSalesforce def _get_source(args: List[str]): @@ -18,9 +18,9 @@ def _get_source(args: List[str]): config_path = AirbyteEntrypoint.extract_config(args) try: return SalesforceSourceDispatcher( - SourceSalesforce( - SourceSalesforce.read_catalog(catalog_path) if catalog_path else None, - SourceSalesforce.read_config(config_path) if config_path else None, + AsyncSourceSalesforce( + AsyncSourceSalesforce.read_catalog(catalog_path) if catalog_path else None, + AsyncSourceSalesforce.read_config(config_path) if config_path else None, ) ) except Exception as error: diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/api.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py similarity index 100% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/api.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/api.py diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py index fffb3cb9e3b1..2624230055a9 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/rate_limiting.py @@ -5,16 +5,17 @@ import logging import sys +import aiohttp import backoff from airbyte_cdk.sources.async_cdk.streams.http.exceptions_async import AsyncDefaultBackoffException from requests import codes, exceptions # type: ignore[import] TRANSIENT_EXCEPTIONS = ( AsyncDefaultBackoffException, - exceptions.ConnectTimeout, - exceptions.ReadTimeout, - exceptions.ConnectionError, - exceptions.HTTPError, + aiohttp.ClientPayloadError, + aiohttp.ServerTimeoutError, + aiohttp.ServerConnectionError, + aiohttp.ServerDisconnectedError, ) logger = logging.getLogger("airbyte") @@ -27,16 +28,16 @@ def log_retry_attempt(details): logger.info(f"Caught retryable error after {details['tries']} tries. Waiting {details['wait']} seconds then retrying...") def should_give_up(exc): - give_up = exc.response is not None and exc.response.status_code != codes.too_many_requests and 400 <= exc.response.status_code < 500 + give_up = exc.response is not None and exc.response.status_code != codes.too_many_requests and 400 <= exc.status_code < 500 # Salesforce can return an error with a limit using a 403 code error. - if exc.response is not None and exc.response.status_code == codes.forbidden: - error_data = exc.response.json()[0] + if exc.status_code == codes.forbidden: + error_data = exc.json()[0] if error_data.get("errorCode", "") == "REQUEST_LIMIT_EXCEEDED": give_up = True if give_up: - logger.info(f"Giving up for returned HTTP status: {exc.response.status_code}, body: {exc.response.text}") + logger.info(f"Giving up for returned HTTP status: {exc.status_code}, body: {exc.text}") return give_up return backoff.on_exception( diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py index cd43bc7f64c3..4d22df1d4fd9 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/source.py @@ -22,7 +22,7 @@ from dateutil.relativedelta import relativedelta from requests import codes, exceptions # type: ignore[import] -from .api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_BULK_API_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce +from source_salesforce.api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_BULK_API_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce from .streams import ( BulkIncrementalSalesforceStream, BulkSalesforceStream, diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py index 51cba51ab0dd..d9d30bfcc1c0 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/streams.py @@ -28,10 +28,10 @@ from requests import codes from requests.models import PreparedRequest -from .api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce -from .availability_strategy import AsyncSalesforceAvailabilityStrategy -from .exceptions import SalesforceException, TmpFileIOError -from .rate_limiting import default_backoff_handler +from source_salesforce.api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce +from source_salesforce.async_salesforce.availability_strategy import AsyncSalesforceAvailabilityStrategy +from source_salesforce.async_salesforce.rate_limiting import default_backoff_handler +from source_salesforce.exceptions import SalesforceException, TmpFileIOError # https://stackoverflow.com/a/54517228 CSV_FIELD_SIZE_LIMIT = int(ctypes.c_ulong(-1).value // 2) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py new file mode 100644 index 000000000000..514727089c2a --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/availability_strategy.py @@ -0,0 +1,35 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +import typing +from typing import Optional, Tuple + +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.http.availability_strategy import HttpAvailabilityStrategy +from requests import HTTPError, codes + +if typing.TYPE_CHECKING: + from airbyte_cdk.sources import Source + + +class SalesforceAvailabilityStrategy(HttpAvailabilityStrategy): + def handle_http_error( + self, stream: Stream, logger: logging.Logger, source: Optional["Source"], error: HTTPError + ) -> Tuple[bool, Optional[str]]: + """ + There are several types of Salesforce sobjects that require additional processing: + 1. Sobjects for which the user, after setting up the data using Airbyte, restricted access, + and we will receive 403 HTTP errors. + 2. There are streams that do not allow you to make a sample using Salesforce `query` or `queryAll`. + And since we use a dynamic method of generating streams for Salesforce connector - at the stage of discover, + we cannot filter out these streams, so we check for them before reading from the streams. + """ + if error.response.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: + error_data = error.response.json()[0] + error_code = error_data.get("errorCode", "") + if error_code != "REQUEST_LIMIT_EXCEEDED" or error_code == "INVALID_TYPE_FOR_OPERATION": + return False, f"Cannot receive data for stream '{stream.name}', error message: '{error_data.get('message')}'" + return True, None + raise error diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/exceptions.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/exceptions.py similarity index 100% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/exceptions.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/exceptions.py diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py new file mode 100644 index 000000000000..344a6412e024 --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/rate_limiting.py @@ -0,0 +1,51 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +import sys + +import backoff +from airbyte_cdk.sources.streams.http.exceptions import DefaultBackoffException +from requests import codes, exceptions # type: ignore[import] + +TRANSIENT_EXCEPTIONS = ( + DefaultBackoffException, + exceptions.ConnectTimeout, + exceptions.ReadTimeout, + exceptions.ConnectionError, + exceptions.HTTPError, +) + +logger = logging.getLogger("airbyte") + + +def default_backoff_handler(max_tries: int, factor: int, **kwargs): + def log_retry_attempt(details): + _, exc, _ = sys.exc_info() + logger.info(str(exc)) + logger.info(f"Caught retryable error after {details['tries']} tries. Waiting {details['wait']} seconds then retrying...") + + def should_give_up(exc): + give_up = exc.response is not None and exc.response.status_code != codes.too_many_requests and 400 <= exc.response.status_code < 500 + + # Salesforce can return an error with a limit using a 403 code error. + if exc.response is not None and exc.response.status_code == codes.forbidden: + error_data = exc.response.json()[0] + if error_data.get("errorCode", "") == "REQUEST_LIMIT_EXCEEDED": + give_up = True + + if give_up: + logger.info(f"Giving up for returned HTTP status: {exc.response.status_code}, body: {exc.response.text}") + return give_up + + return backoff.on_exception( + backoff.expo, + TRANSIENT_EXCEPTIONS, + jitter=None, + on_backoff=log_retry_attempt, + giveup=should_give_up, + max_tries=max_tries, + factor=factor, + **kwargs, + ) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py new file mode 100644 index 000000000000..30eea954dfe0 --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/source.py @@ -0,0 +1,243 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import logging +from datetime import datetime +from typing import Any, Iterator, List, Mapping, MutableMapping, Optional, Tuple, Union + +import requests +from airbyte_cdk import AirbyteLogger +from airbyte_cdk.logger import AirbyteLogFormatter +from airbyte_cdk.models import AirbyteMessage, AirbyteStateMessage, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, Level, SyncMode +from airbyte_cdk.sources.concurrent_source.concurrent_source import ConcurrentSource +from airbyte_cdk.sources.concurrent_source.concurrent_source_adapter import ConcurrentSourceAdapter +from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager +from airbyte_cdk.sources.message import InMemoryMessageRepository +from airbyte_cdk.sources.streams import Stream +from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade +from airbyte_cdk.sources.streams.concurrent.cursor import NoopCursor +from airbyte_cdk.sources.streams.http.auth import TokenAuthenticator +from airbyte_cdk.sources.utils.schema_helpers import InternalConfig +from airbyte_cdk.utils.traced_exception import AirbyteTracedException +from dateutil.relativedelta import relativedelta +from requests import codes, exceptions # type: ignore[import] + +from .api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_BULK_API_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce +from .streams import ( + BulkIncrementalSalesforceStream, + BulkSalesforceStream, + BulkSalesforceSubStream, + Describe, + IncrementalRestSalesforceStream, + RestSalesforceStream, + RestSalesforceSubStream, +) + +_DEFAULT_CONCURRENCY = 10 +_MAX_CONCURRENCY = 10 +logger = logging.getLogger("airbyte") + + +class AirbyteStopSync(AirbyteTracedException): + pass + + +class SourceSalesforce(ConcurrentSourceAdapter): + DATETIME_FORMAT = "%Y-%m-%dT%H:%M:%SZ" + START_DATE_OFFSET_IN_YEARS = 2 + MAX_WORKERS = 5 + + message_repository = InMemoryMessageRepository(Level(AirbyteLogFormatter.level_mapping[logger.level])) + + def __init__(self, catalog: Optional[ConfiguredAirbyteCatalog], config: Optional[Mapping[str, Any]], **kwargs): + if config: + concurrency_level = min(config.get("num_workers", _DEFAULT_CONCURRENCY), _MAX_CONCURRENCY) + else: + concurrency_level = _DEFAULT_CONCURRENCY + logger.info(f"Using concurrent cdk with concurrency level {concurrency_level}") + concurrent_source = ConcurrentSource.create( + concurrency_level, concurrency_level // 2, logger, self._slice_logger, self.message_repository + ) + super().__init__(concurrent_source) + self.catalog = catalog + + @staticmethod + def _get_sf_object(config: Mapping[str, Any]) -> Salesforce: + sf = Salesforce(**config) + sf.login() + return sf + + def check_connection(self, logger: AirbyteLogger, config: Mapping[str, Any]) -> Tuple[bool, Optional[str]]: + try: + salesforce = self._get_sf_object(config) + salesforce.describe() + except exceptions.HTTPError as error: + error_msg = f"An error occurred: {error.response.text}" + try: + error_data = error.response.json()[0] + except (KeyError, requests.exceptions.JSONDecodeError): + pass + else: + error_code = error_data.get("errorCode") + if error.response.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + logger.warn(f"API Call limit is exceeded. Error message: '{error_data.get('message')}'") + error_msg = "API Call limit is exceeded" + return False, error_msg + return True, None + + @classmethod + def _get_api_type(cls, stream_name: str, json_schema: Mapping[str, Any], force_use_bulk_api: bool) -> str: + """Get proper API type: rest or bulk""" + # Salesforce BULK API currently does not support loading fields with data type base64 and compound data + properties = json_schema.get("properties", {}) + properties_not_supported_by_bulk = { + key: value for key, value in properties.items() if value.get("format") == "base64" or "object" in value["type"] + } + rest_only = stream_name in UNSUPPORTED_BULK_API_SALESFORCE_OBJECTS + if rest_only: + logger.warning(f"BULK API is not supported for stream: {stream_name}") + return "rest" + if force_use_bulk_api and properties_not_supported_by_bulk: + logger.warning( + f"Following properties will be excluded from stream: {stream_name} due to BULK API limitations: {list(properties_not_supported_by_bulk)}" + ) + return "bulk" + if properties_not_supported_by_bulk: + return "rest" + return "bulk" + + @classmethod + def _get_stream_type(cls, stream_name: str, api_type: str): + """Get proper stream class: full_refresh, incremental or substream + + SubStreams (like ContentDocumentLink) do not support incremental sync because of query restrictions, look here: + https://developer.salesforce.com/docs/atlas.en-us.object_reference.meta/object_reference/sforce_api_objects_contentdocumentlink.htm + """ + parent_name = PARENT_SALESFORCE_OBJECTS.get(stream_name, {}).get("parent_name") + if api_type == "rest": + full_refresh = RestSalesforceSubStream if parent_name else RestSalesforceStream + incremental = IncrementalRestSalesforceStream + elif api_type == "bulk": + full_refresh = BulkSalesforceSubStream if parent_name else BulkSalesforceStream + incremental = BulkIncrementalSalesforceStream + else: + raise Exception(f"Stream {stream_name} cannot be processed by REST or BULK API.") + return full_refresh, incremental + + @classmethod + def prepare_stream(cls, stream_name: str, json_schema, sobject_options, sf_object, authenticator, config): + """Choose proper stream class: syncMode(full_refresh/incremental), API type(Rest/Bulk), SubStream""" + pk, replication_key = sf_object.get_pk_and_replication_key(json_schema) + stream_kwargs = { + "stream_name": stream_name, + "schema": json_schema, + "pk": pk, + "sobject_options": sobject_options, + "sf_api": sf_object, + "authenticator": authenticator, + "start_date": config.get("start_date"), + } + + api_type = cls._get_api_type(stream_name, json_schema, config.get("force_use_bulk_api", False)) + full_refresh, incremental = cls._get_stream_type(stream_name, api_type) + if replication_key and stream_name not in UNSUPPORTED_FILTERING_STREAMS: + stream_class = incremental + stream_kwargs["replication_key"] = replication_key + else: + stream_class = full_refresh + + return stream_class, stream_kwargs + + @classmethod + def generate_streams( + cls, + config: Mapping[str, Any], + stream_objects: Mapping[str, Any], + sf_object: Salesforce, + ) -> List[Stream]: + """Generates a list of stream by their names. It can be used for different tests too""" + authenticator = TokenAuthenticator(sf_object.access_token) + schemas = sf_object.generate_schemas(stream_objects) + default_args = [sf_object, authenticator, config] + streams = [] + for stream_name, sobject_options in stream_objects.items(): + json_schema = schemas.get(stream_name, {}) + + stream_class, kwargs = cls.prepare_stream(stream_name, json_schema, sobject_options, *default_args) + + parent_name = PARENT_SALESFORCE_OBJECTS.get(stream_name, {}).get("parent_name") + if parent_name: + # get minimal schema required for getting proper class name full_refresh/incremental, rest/bulk + parent_schema = PARENT_SALESFORCE_OBJECTS.get(stream_name, {}).get("schema_minimal") + parent_class, parent_kwargs = cls.prepare_stream(parent_name, parent_schema, sobject_options, *default_args) + kwargs["parent"] = parent_class(**parent_kwargs) + + stream = stream_class(**kwargs) + + api_type = cls._get_api_type(stream_name, json_schema, config.get("force_use_bulk_api", False)) + if api_type == "rest" and not stream.primary_key and stream.too_many_properties: + logger.warning( + f"Can not instantiate stream {stream_name}. It is not supported by the BULK API and can not be " + "implemented via REST because the number of its properties exceeds the limit and it lacks a primary key." + ) + continue + streams.append(stream) + return streams + + def streams(self, config: Mapping[str, Any]) -> List[Stream]: + if not config.get("start_date"): + config["start_date"] = (datetime.now() - relativedelta(years=self.START_DATE_OFFSET_IN_YEARS)).strftime(self.DATETIME_FORMAT) + sf = self._get_sf_object(config) + stream_objects = sf.get_validated_streams(config=config, catalog=self.catalog) + streams = self.generate_streams(config, stream_objects, sf) + streams.append(Describe(sf_api=sf, catalog=self.catalog)) + # TODO: incorporate state & ConcurrentCursor when we support incremental + configured_streams = [] + for stream in streams: + sync_mode = self._get_sync_mode_from_catalog(stream) + if sync_mode == SyncMode.full_refresh: + configured_streams.append(StreamFacade.create_from_stream(stream, self, logger, None, NoopCursor())) + else: + configured_streams.append(stream) + return configured_streams + + def _get_sync_mode_from_catalog(self, stream: Stream) -> Optional[SyncMode]: + if self.catalog: + for catalog_stream in self.catalog.streams: + if stream.name == catalog_stream.stream.name: + return catalog_stream.sync_mode + return None + + def read( + self, + logger: logging.Logger, + config: Mapping[str, Any], + catalog: ConfiguredAirbyteCatalog, + state: Union[List[AirbyteStateMessage], MutableMapping[str, Any]] = None, + ) -> Iterator[AirbyteMessage]: + # save for use inside streams method + self.catalog = catalog + try: + yield from super().read(logger, config, catalog, state) + except AirbyteStopSync: + logger.info(f"Finished syncing {self.name}") + + def _read_stream( + self, + logger: logging.Logger, + stream_instance: Stream, + configured_stream: ConfiguredAirbyteStream, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ) -> Iterator[AirbyteMessage]: + try: + yield from super()._read_stream(logger, stream_instance, configured_stream, state_manager, internal_config) + except exceptions.HTTPError as error: + error_data = error.response.json()[0] + error_code = error_data.get("errorCode") + url = error.response.url + if error.response.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + logger.warning(f"API Call {url} limit is exceeded. Error message: '{error_data.get('message')}'") + raise AirbyteStopSync() # if got 403 rate limit response, finish the sync with success. + raise error diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py new file mode 100644 index 000000000000..34c03d1caa94 --- /dev/null +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/streams.py @@ -0,0 +1,790 @@ +# +# Copyright (c) 2023 Airbyte, Inc., all rights reserved. +# + +import csv +import ctypes +import math +import os +import time +import urllib.parse +import uuid +from abc import ABC +from contextlib import closing +from typing import Any, Callable, Iterable, List, Mapping, MutableMapping, Optional, Tuple, Type, Union + +import pandas as pd +import pendulum +import requests # type: ignore[import] +from airbyte_cdk.models import ConfiguredAirbyteCatalog, FailureType, SyncMode +from airbyte_cdk.sources.streams.availability_strategy import AvailabilityStrategy +from airbyte_cdk.sources.streams.core import Stream, StreamData +from airbyte_cdk.sources.streams.http import HttpStream, HttpSubStream +from airbyte_cdk.sources.utils.transform import TransformConfig, TypeTransformer +from airbyte_cdk.utils import AirbyteTracedException +from numpy import nan +from pendulum import DateTime # type: ignore[attr-defined] +from requests import codes, exceptions +from requests.models import PreparedRequest + +from .api import PARENT_SALESFORCE_OBJECTS, UNSUPPORTED_FILTERING_STREAMS, Salesforce +from .availability_strategy import SalesforceAvailabilityStrategy +from .exceptions import SalesforceException, TmpFileIOError +from .rate_limiting import default_backoff_handler + +# https://stackoverflow.com/a/54517228 +CSV_FIELD_SIZE_LIMIT = int(ctypes.c_ulong(-1).value // 2) +csv.field_size_limit(CSV_FIELD_SIZE_LIMIT) + +DEFAULT_ENCODING = "utf-8" + + +class SalesforceStream(HttpStream, ABC): + page_size = 2000 + transformer = TypeTransformer(TransformConfig.DefaultSchemaNormalization) + encoding = DEFAULT_ENCODING + + def __init__( + self, + sf_api: Salesforce, + pk: str, + stream_name: str, + sobject_options: Mapping[str, Any] = None, + schema: dict = None, + start_date=None, + **kwargs, + ): + super().__init__(**kwargs) + self.sf_api = sf_api + self.pk = pk + self.stream_name = stream_name + self.schema: Mapping[str, Any] = schema # type: ignore[assignment] + self.sobject_options = sobject_options + self.start_date = self.format_start_date(start_date) + + @staticmethod + def format_start_date(start_date: Optional[str]) -> Optional[str]: + """Transform the format `2021-07-25` into the format `2021-07-25T00:00:00Z`""" + if start_date: + return pendulum.parse(start_date).strftime("%Y-%m-%dT%H:%M:%SZ") # type: ignore[attr-defined,no-any-return] + return None + + @property + def max_properties_length(self) -> int: + return Salesforce.REQUEST_SIZE_LIMITS - len(self.url_base) - 2000 + + @property + def name(self) -> str: + return self.stream_name + + @property + def primary_key(self) -> Optional[Union[str, List[str], List[List[str]]]]: + return self.pk + + @property + def url_base(self) -> str: + return self.sf_api.instance_url + + @property + def availability_strategy(self) -> Optional["AvailabilityStrategy"]: + return SalesforceAvailabilityStrategy() + + @property + def too_many_properties(self): + selected_properties = self.get_json_schema().get("properties", {}) + properties_length = len(urllib.parse.quote(",".join(p for p in selected_properties))) + return properties_length > self.max_properties_length + + def parse_response(self, response: requests.Response, **kwargs) -> Iterable[Mapping]: + yield from response.json()["records"] + + def get_json_schema(self) -> Mapping[str, Any]: + if not self.schema: + self.schema = self.sf_api.generate_schema(self.name) + return self.schema + + def get_error_display_message(self, exception: BaseException) -> Optional[str]: + if isinstance(exception, exceptions.ConnectionError): + return f"After {self.max_retries} retries the connector has failed with a network error. It looks like Salesforce API experienced temporary instability, please try again later." + return super().get_error_display_message(exception) + + +class PropertyChunk: + """ + Object that is used to keep track of the current state of a chunk of properties for the stream of records being synced. + """ + + properties: Mapping[str, Any] + first_time: bool + record_counter: int + next_page: Optional[Mapping[str, Any]] + + def __init__(self, properties: Mapping[str, Any]): + self.properties = properties + self.first_time = True + self.record_counter = 0 + self.next_page = None + + +class RestSalesforceStream(SalesforceStream): + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + assert self.primary_key or not self.too_many_properties + + def path(self, next_page_token: Mapping[str, Any] = None, **kwargs: Any) -> str: + if next_page_token: + """ + If `next_page_token` is set, subsequent requests use `nextRecordsUrl`. + """ + next_token: str = next_page_token["next_token"] + return next_token + return f"/services/data/{self.sf_api.version}/queryAll" + + def next_page_token(self, response: requests.Response) -> Optional[Mapping[str, Any]]: + response_data = response.json() + next_token = response_data.get("nextRecordsUrl") + return {"next_token": next_token} if next_token else None + + def request_params( + self, + stream_state: Mapping[str, Any], + stream_slice: Mapping[str, Any] = None, + next_page_token: Mapping[str, Any] = None, + property_chunk: Mapping[str, Any] = None, + ) -> MutableMapping[str, Any]: + """ + Salesforce SOQL Query: https://developer.salesforce.com/docs/atlas.en-us.232.0.api_rest.meta/api_rest/dome_queryall.htm + """ + if next_page_token: + # If `next_page_token` is set, subsequent requests use `nextRecordsUrl`, and do not include any parameters. + return {} + + property_chunk = property_chunk or {} + query = f"SELECT {','.join(property_chunk.keys())} FROM {self.name} " + + if self.name in PARENT_SALESFORCE_OBJECTS: + # add where clause: " WHERE ContentDocumentId IN ('06905000000NMXXXXX', ...)" + parent_field = PARENT_SALESFORCE_OBJECTS[self.name]["field"] + parent_ids = [f"'{parent_record[parent_field]}'" for parent_record in stream_slice["parents"]] + query += f" WHERE ContentDocumentId IN ({','.join(parent_ids)})" + + if self.primary_key and self.name not in UNSUPPORTED_FILTERING_STREAMS: + query += f"ORDER BY {self.primary_key} ASC" + + return {"q": query} + + def chunk_properties(self) -> Iterable[Mapping[str, Any]]: + selected_properties = self.get_json_schema().get("properties", {}) + + def empty_props_with_pk_if_present(): + return {self.primary_key: selected_properties[self.primary_key]} if self.primary_key else {} + + summary_length = 0 + local_properties = empty_props_with_pk_if_present() + for property_name, value in selected_properties.items(): + current_property_length = len(urllib.parse.quote(f"{property_name},")) + if current_property_length + summary_length >= self.max_properties_length: + yield local_properties + local_properties = empty_props_with_pk_if_present() + summary_length = 0 + + local_properties[property_name] = value + summary_length += current_property_length + + if local_properties: + yield local_properties + + @staticmethod + def _next_chunk_id(property_chunks: Mapping[int, PropertyChunk]) -> Optional[int]: + """ + Figure out which chunk is going to be read next. + It should be the one with the least number of records read by the moment. + """ + non_exhausted_chunks = { + # We skip chunks that have already attempted a sync before and do not have a next page + chunk_id: property_chunk.record_counter + for chunk_id, property_chunk in property_chunks.items() + if property_chunk.first_time or property_chunk.next_page + } + if not non_exhausted_chunks: + return None + return min(non_exhausted_chunks, key=non_exhausted_chunks.get) + + def _read_pages( + self, + records_generator_fn: Callable[ + [requests.PreparedRequest, requests.Response, Mapping[str, Any], Mapping[str, Any]], Iterable[StreamData] + ], + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ) -> Iterable[StreamData]: + stream_state = stream_state or {} + records_by_primary_key = {} + property_chunks: Mapping[int, PropertyChunk] = { + index: PropertyChunk(properties=properties) for index, properties in enumerate(self.chunk_properties()) + } + while True: + chunk_id = self._next_chunk_id(property_chunks) + if chunk_id is None: + # pagination complete + break + + property_chunk = property_chunks[chunk_id] + request, response = self._fetch_next_page_for_chunk( + stream_slice, stream_state, property_chunk.next_page, property_chunk.properties + ) + + # When this is the first time we're getting a chunk's records, we set this to False to be used when deciding the next chunk + if property_chunk.first_time: + property_chunk.first_time = False + property_chunk.next_page = self.next_page_token(response) + chunk_page_records = records_generator_fn(request, response, stream_state, stream_slice) + if not self.too_many_properties: + # this is the case when a stream has no primary key + # (it is allowed when properties length does not exceed the maximum value) + # so there would be a single chunk, therefore we may and should yield records immediately + for record in chunk_page_records: + property_chunk.record_counter += 1 + yield record + continue + + # stick together different parts of records by their primary key and emit if a record is complete + for record in chunk_page_records: + property_chunk.record_counter += 1 + record_id = record[self.primary_key] + if record_id not in records_by_primary_key: + records_by_primary_key[record_id] = (record, 1) + continue + partial_record, counter = records_by_primary_key[record_id] + partial_record.update(record) + counter += 1 + if counter == len(property_chunks): + yield partial_record # now it's complete + records_by_primary_key.pop(record_id) + else: + records_by_primary_key[record_id] = (partial_record, counter) + + # Process what's left. + # Because we make multiple calls to query N records (each call to fetch X properties of all the N records), + # there's a chance that the number of records corresponding to the query may change between the calls. + # Select 'a', 'b' from table order by pk -> returns records with ids `1`, `2` + # + # Select 'c', 'd' from table order by pk -> returns records with ids `1`, `3` + # Then records `2` and `3` would be incomplete. + # This may result in data inconsistency. We skip such records for now and log a warning message. + incomplete_record_ids = ",".join([str(key) for key in records_by_primary_key]) + if incomplete_record_ids: + self.logger.warning(f"Inconsistent record(s) with primary keys {incomplete_record_ids} found. Skipping them.") + + # Always return an empty generator just in case no records were ever yielded + yield from [] + + def _fetch_next_page_for_chunk( + self, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + next_page_token: Mapping[str, Any] = None, + property_chunk: Mapping[str, Any] = None, + ) -> Tuple[requests.PreparedRequest, requests.Response]: + request_headers = self.request_headers(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + request = self._create_prepared_request( + path=self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + headers=dict(request_headers, **self.authenticator.get_auth_header()), + params=self.request_params( + stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token, property_chunk=property_chunk + ), + json=self.request_body_json(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + data=self.request_body_data(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token), + ) + request_kwargs = self.request_kwargs(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + response = self._send_request(request, request_kwargs) + return request, response + + +class BatchedSubStream(HttpSubStream): + SLICE_BATCH_SIZE = 200 + + def stream_slices( + self, sync_mode: SyncMode, cursor_field: Optional[List[str]] = None, stream_state: Optional[Mapping[str, Any]] = None + ) -> Iterable[Optional[Mapping[str, Any]]]: + """Instead of yielding one parent record at a time, make stream slice contain a batch of parent records. + + It allows to get records by one requests (instead of only one). + """ + batched_slice = [] + for stream_slice in super().stream_slices(sync_mode, cursor_field, stream_state): + if len(batched_slice) == self.SLICE_BATCH_SIZE: + yield {"parents": batched_slice} + batched_slice = [] + batched_slice.append(stream_slice["parent"]) + if batched_slice: + yield {"parents": batched_slice} + + +class RestSalesforceSubStream(BatchedSubStream, RestSalesforceStream): + pass + + +class BulkSalesforceStream(SalesforceStream): + DEFAULT_WAIT_TIMEOUT_SECONDS = 86400 # 24-hour bulk job running time + MAX_CHECK_INTERVAL_SECONDS = 2.0 + MAX_RETRY_NUMBER = 3 + + def path(self, next_page_token: Mapping[str, Any] = None, **kwargs: Any) -> str: + return f"/services/data/{self.sf_api.version}/jobs/query" + + transformer = TypeTransformer(TransformConfig.CustomSchemaNormalization | TransformConfig.DefaultSchemaNormalization) + + @default_backoff_handler(max_tries=5, factor=15) + def _send_http_request(self, method: str, url: str, json: dict = None, headers: dict = None, stream: bool = False): + headers = self.authenticator.get_auth_header() if not headers else headers | self.authenticator.get_auth_header() + response = self._session.request(method, url=url, headers=headers, json=json, stream=stream) + if response.status_code not in [200, 204]: + self.logger.error(f"error body: {response.text}, sobject options: {self.sobject_options}") + response.raise_for_status() + return response + + def create_stream_job(self, query: str, url: str) -> Optional[str]: + """ + docs: https://developer.salesforce.com/docs/atlas.en-us.api_asynch.meta/api_asynch/create_job.html + """ + json = {"operation": "queryAll", "query": query, "contentType": "CSV", "columnDelimiter": "COMMA", "lineEnding": "LF"} + try: + response = self._send_http_request("POST", url, json=json) + job_id: str = response.json()["id"] + return job_id + except exceptions.HTTPError as error: + if error.response.status_code in [codes.FORBIDDEN, codes.BAD_REQUEST]: + # A part of streams can't be used by BULK API. Every API version can have a custom list of + # these sobjects. Another part of them can be generated dynamically. That's why we can't track + # them preliminarily and there is only one way is to except error with necessary messages about + # their limitations. Now we know about 3 different reasons of similar errors: + # 1) some SaleForce sobjects(streams) is not supported by the BULK API simply (as is). + # 2) Access to a sobject(stream) is not available + # 3) sobject is not queryable. It means this sobject can't be called directly. + # We can call it as part of response from another sobject only. E.g.: + # initial query: "Select Id, Subject from ActivityHistory" -> error + # updated query: "Select Name, (Select Subject,ActivityType from ActivityHistories) from Contact" + # The second variant forces customisation for every case (ActivityHistory, ActivityHistories etc). + # And the main problem is these subqueries doesn't support CSV response format. + error_data = error.response.json()[0] + error_code = error_data.get("errorCode") + error_message = error_data.get("message", "") + if error_message == "Selecting compound data not supported in Bulk Query" or ( + error_code == "INVALIDENTITY" and "is not supported by the Bulk API" in error_message + ): + self.logger.error( + f"Cannot receive data for stream '{self.name}' using BULK API, " + f"sobject options: {self.sobject_options}, error message: '{error_message}'" + ) + elif error.response.status_code == codes.FORBIDDEN and error_code != "REQUEST_LIMIT_EXCEEDED": + self.logger.error( + f"Cannot receive data for stream '{self.name}' ," + f"sobject options: {self.sobject_options}, error message: '{error_message}'" + ) + elif error.response.status_code == codes.FORBIDDEN and error_code == "REQUEST_LIMIT_EXCEEDED": + self.logger.error( + f"Cannot receive data for stream '{self.name}' ," + f"sobject options: {self.sobject_options}, Error message: '{error_data.get('message')}'" + ) + elif error.response.status_code == codes.BAD_REQUEST and error_message.endswith("does not support query"): + self.logger.error( + f"The stream '{self.name}' is not queryable, " + f"sobject options: {self.sobject_options}, error message: '{error_message}'" + ) + elif ( + error.response.status_code == codes.BAD_REQUEST + and error_code == "API_ERROR" + and error_message.startswith("Implementation restriction") + ): + message = f"Unable to sync '{self.name}'. To prevent future syncs from failing, ensure the authenticated user has \"View all Data\" permissions." + raise AirbyteTracedException(message=message, failure_type=FailureType.config_error, exception=error) + elif error.response.status_code == codes.BAD_REQUEST and error_code == "LIMIT_EXCEEDED": + message = "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed." + self.logger.error(message) + else: + raise error + else: + raise error + return None + + def wait_for_job(self, url: str) -> str: + expiration_time: DateTime = pendulum.now().add(seconds=self.DEFAULT_WAIT_TIMEOUT_SECONDS) + job_status = "InProgress" + delay_timeout = 0.0 + delay_cnt = 0 + job_info = None + # minimal starting delay is 0.5 seconds. + # this value was received empirically + time.sleep(0.5) + while pendulum.now() < expiration_time: + try: + job_info = self._send_http_request("GET", url=url).json() + except exceptions.HTTPError as error: + error_data = error.response.json()[0] + error_code = error_data.get("errorCode") + error_message = error_data.get("message", "") + if ( + "We can't complete the action because enabled transaction security policies took too long to complete." in error_message + and error_code == "TXN_SECURITY_METERING_ERROR" + ): + message = 'A transient authentication error occurred. To prevent future syncs from failing, assign the "Exempt from Transaction Security" user permission to the authenticated user.' + raise AirbyteTracedException(message=message, failure_type=FailureType.config_error, exception=error) + else: + raise error + job_status = job_info["state"] + if job_status in ["JobComplete", "Aborted", "Failed"]: + if job_status != "JobComplete": + # this is only job metadata without payload + error_message = job_info.get("errorMessage") + if not error_message: + # not all failed response can have "errorMessage" and we need to show full response body + error_message = job_info + self.logger.error(f"JobStatus: {job_status}, sobject options: {self.sobject_options}, error message: '{error_message}'") + + return job_status + + if delay_timeout < self.MAX_CHECK_INTERVAL_SECONDS: + delay_timeout = 0.5 + math.exp(delay_cnt) / 1000.0 + delay_cnt += 1 + + time.sleep(delay_timeout) + job_id = job_info["id"] + self.logger.info( + f"Sleeping {delay_timeout} seconds while waiting for Job: {self.name}/{job_id} to complete. Current state: {job_status}" + ) + + self.logger.warning(f"Not wait the {self.name} data for {self.DEFAULT_WAIT_TIMEOUT_SECONDS} seconds, data: {job_info}!!") + return job_status + + def execute_job(self, query: str, url: str) -> Tuple[Optional[str], Optional[str]]: + job_status = "Failed" + for i in range(0, self.MAX_RETRY_NUMBER): + job_id = self.create_stream_job(query=query, url=url) + if not job_id: + return None, job_status + job_full_url = f"{url}/{job_id}" + job_status = self.wait_for_job(url=job_full_url) + if job_status not in ["UploadComplete", "InProgress"]: + break + self.logger.error(f"Waiting error. Try to run this job again {i + 1}/{self.MAX_RETRY_NUMBER}...") + self.abort_job(url=job_full_url) + job_status = "Aborted" + + if job_status in ["Aborted", "Failed"]: + self.delete_job(url=job_full_url) + return None, job_status + return job_full_url, job_status + + def filter_null_bytes(self, b: bytes): + """ + https://github.com/airbytehq/airbyte/issues/8300 + """ + res = b.replace(b"\x00", b"") + if len(res) < len(b): + self.logger.warning("Filter 'null' bytes from string, size reduced %d -> %d chars", len(b), len(res)) + return res + + def get_response_encoding(self, headers) -> str: + """Returns encodings from given HTTP Header Dict. + + :param headers: dictionary to extract encoding from. + :rtype: str + """ + + content_type = headers.get("content-type") + + if not content_type: + return self.encoding + + content_type, params = requests.utils._parse_content_type_header(content_type) + + if "charset" in params: + return params["charset"].strip("'\"") + + return self.encoding + + def download_data(self, url: str, chunk_size: int = 1024) -> tuple[str, str, dict]: + """ + Retrieves binary data result from successfully `executed_job`, using chunks, to avoid local memory limitations. + @ url: string - the url of the `executed_job` + @ chunk_size: int - the buffer size for each chunk to fetch from stream, in bytes, default: 1024 bytes + Return the tuple containing string with file path of downloaded binary data (Saved temporarily) and file encoding. + """ + # set filepath for binary data from response + tmp_file = str(uuid.uuid4()) + with closing(self._send_http_request("GET", url, headers={"Accept-Encoding": "gzip"}, stream=True)) as response, open( + tmp_file, "wb" + ) as data_file: + response_headers = response.headers + response_encoding = self.get_response_encoding(response_headers) + for chunk in response.iter_content(chunk_size=chunk_size): + data_file.write(self.filter_null_bytes(chunk)) + # check the file exists + if os.path.isfile(tmp_file): + return tmp_file, response_encoding, response_headers + else: + raise TmpFileIOError(f"The IO/Error occured while verifying binary data. Stream: {self.name}, file {tmp_file} doesn't exist.") + + def read_with_chunks(self, path: str, file_encoding: str, chunk_size: int = 100) -> Iterable[Tuple[int, Mapping[str, Any]]]: + """ + Reads the downloaded binary data, using lines chunks, set by `chunk_size`. + @ path: string - the path to the downloaded temporarily binary data. + @ file_encoding: string - encoding for binary data file according to Standard Encodings from codecs module + @ chunk_size: int - the number of lines to read at a time, default: 100 lines / time. + """ + try: + with open(path, "r", encoding=file_encoding) as data: + chunks = pd.read_csv(data, chunksize=chunk_size, iterator=True, dialect="unix", dtype=object) + for chunk in chunks: + chunk = chunk.replace({nan: None}).to_dict(orient="records") + for row in chunk: + yield row + except pd.errors.EmptyDataError as e: + self.logger.info(f"Empty data received. {e}") + yield from [] + except IOError as ioe: + raise TmpFileIOError(f"The IO/Error occured while reading tmp data. Called: {path}. Stream: {self.name}", ioe) + finally: + # remove binary tmp file, after data is read + os.remove(path) + + def abort_job(self, url: str): + data = {"state": "Aborted"} + self._send_http_request("PATCH", url=url, json=data) + self.logger.warning("Broken job was aborted") + + def delete_job(self, url: str): + self._send_http_request("DELETE", url=url) + + @property + def availability_strategy(self) -> Optional["AvailabilityStrategy"]: + return None + + def next_page_token(self, last_record: Mapping[str, Any]) -> Optional[Mapping[str, Any]]: + return None + + def get_query_select_fields(self) -> str: + return ", ".join( + { + key: value + for key, value in self.get_json_schema().get("properties", {}).items() + if value.get("format") != "base64" and "object" not in value["type"] + } + ) + + def request_params( + self, stream_state: Mapping[str, Any], stream_slice: Mapping[str, Any] = None, next_page_token: Mapping[str, Any] = None + ) -> MutableMapping[str, Any]: + """ + Salesforce SOQL Query: https://developer.salesforce.com/docs/atlas.en-us.232.0.api_rest.meta/api_rest/dome_queryall.htm + """ + + select_fields = self.get_query_select_fields() + query = f"SELECT {select_fields} FROM {self.name}" + if next_page_token: + query += next_page_token["next_token"] + + if self.name in PARENT_SALESFORCE_OBJECTS: + # add where clause: " WHERE ContentDocumentId IN ('06905000000NMXXXXX', '06905000000Mxp7XXX', ...)" + parent_field = PARENT_SALESFORCE_OBJECTS[self.name]["field"] + parent_ids = [f"'{parent_record[parent_field]}'" for parent_record in stream_slice["parents"]] + query += f" WHERE ContentDocumentId IN ({','.join(parent_ids)})" + + return {"q": query} + + def read_records( + self, + sync_mode: SyncMode, + cursor_field: List[str] = None, + stream_slice: Mapping[str, Any] = None, + stream_state: Mapping[str, Any] = None, + ) -> Iterable[Mapping[str, Any]]: + stream_state = stream_state or {} + next_page_token = None + + params = self.request_params(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + path = self.path(stream_state=stream_state, stream_slice=stream_slice, next_page_token=next_page_token) + job_full_url, job_status = self.execute_job(query=params["q"], url=f"{self.url_base}{path}") + if not job_full_url: + if job_status == "Failed": + # As rule as BULK logic returns unhandled error. For instance: + # error message: 'Unexpected exception encountered in query processing. + # Please contact support with the following id: 326566388-63578 (-436445966)'" + # Thus we can try to switch to GET sync request because its response returns obvious error message + standard_instance = self.get_standard_instance() + self.logger.warning("switch to STANDARD(non-BULK) sync. Because the SalesForce BULK job has returned a failed status") + stream_is_available, error = standard_instance.check_availability(self.logger, None) + if not stream_is_available: + self.logger.warning(f"Skipped syncing stream '{standard_instance.name}' because it was unavailable. Error: {error}") + return + yield from standard_instance.read_records( + sync_mode=sync_mode, cursor_field=cursor_field, stream_slice=stream_slice, stream_state=stream_state + ) + return + raise SalesforceException(f"Job for {self.name} stream using BULK API was failed.") + salesforce_bulk_api_locator = None + while True: + req = PreparedRequest() + req.prepare_url(f"{job_full_url}/results", {"locator": salesforce_bulk_api_locator}) + tmp_file, response_encoding, response_headers = self.download_data(url=req.url) + for record in self.read_with_chunks(tmp_file, response_encoding): + yield record + + if response_headers.get("Sforce-Locator", "null") == "null": + break + salesforce_bulk_api_locator = response_headers.get("Sforce-Locator") + self.delete_job(url=job_full_url) + + def get_standard_instance(self) -> SalesforceStream: + """Returns a instance of standard logic(non-BULK) with same settings""" + stream_kwargs = dict( + sf_api=self.sf_api, + pk=self.pk, + stream_name=self.stream_name, + schema=self.schema, + sobject_options=self.sobject_options, + authenticator=self.authenticator, + ) + new_cls: Type[SalesforceStream] = RestSalesforceStream + if isinstance(self, BulkIncrementalSalesforceStream): + stream_kwargs.update({"replication_key": self.replication_key, "start_date": self.start_date}) + new_cls = IncrementalRestSalesforceStream + + return new_cls(**stream_kwargs) + + +class BulkSalesforceSubStream(BatchedSubStream, BulkSalesforceStream): + pass + + +@BulkSalesforceStream.transformer.registerCustomTransform +def transform_empty_string_to_none(instance: Any, schema: Any): + """ + BULK API returns a `csv` file, where all values are initially as string type. + This custom transformer replaces empty lines with `None` value. + """ + if isinstance(instance, str) and not instance.strip(): + instance = None + + return instance + + +class IncrementalRestSalesforceStream(RestSalesforceStream, ABC): + state_checkpoint_interval = 500 + STREAM_SLICE_STEP = 30 + _slice = None + + def __init__(self, replication_key: str, **kwargs): + super().__init__(**kwargs) + self.replication_key = replication_key + + def stream_slices( + self, *, sync_mode: SyncMode, cursor_field: List[str] = None, stream_state: Mapping[str, Any] = None + ) -> Iterable[Optional[Mapping[str, Any]]]: + start, end = (None, None) + now = pendulum.now(tz="UTC") + initial_date = pendulum.parse((stream_state or {}).get(self.cursor_field, self.start_date), tz="UTC") + + slice_number = 1 + while not end == now: + start = initial_date.add(days=(slice_number - 1) * self.STREAM_SLICE_STEP) + end = min(now, initial_date.add(days=slice_number * self.STREAM_SLICE_STEP)) + self._slice = {"start_date": start.isoformat(timespec="milliseconds"), "end_date": end.isoformat(timespec="milliseconds")} + yield {"start_date": start.isoformat(timespec="milliseconds"), "end_date": end.isoformat(timespec="milliseconds")} + slice_number = slice_number + 1 + + def request_params( + self, + stream_state: Mapping[str, Any], + stream_slice: Mapping[str, Any] = None, + next_page_token: Mapping[str, Any] = None, + property_chunk: Mapping[str, Any] = None, + ) -> MutableMapping[str, Any]: + if next_page_token: + """ + If `next_page_token` is set, subsequent requests use `nextRecordsUrl`, and do not include any parameters. + """ + return {} + + property_chunk = property_chunk or {} + + start_date = max( + (stream_state or {}).get(self.cursor_field, self.start_date), + (stream_slice or {}).get("start_date", ""), + (next_page_token or {}).get("start_date", ""), + ) + end_date = (stream_slice or {}).get("end_date", pendulum.now(tz="UTC").isoformat(timespec="milliseconds")) + + select_fields = ",".join(property_chunk.keys()) + table_name = self.name + where_conditions = [] + + if start_date: + where_conditions.append(f"{self.cursor_field} >= {start_date}") + if end_date: + where_conditions.append(f"{self.cursor_field} < {end_date}") + + where_clause = f"WHERE {' AND '.join(where_conditions)}" + query = f"SELECT {select_fields} FROM {table_name} {where_clause}" + + return {"q": query} + + @property + def cursor_field(self) -> str: + return self.replication_key + + def get_updated_state(self, current_stream_state: MutableMapping[str, Any], latest_record: Mapping[str, Any]) -> Mapping[str, Any]: + """ + Return the latest state by comparing the cursor value in the latest record with the stream's most recent state + object and returning an updated state object. Check if latest record is IN stream slice interval => ignore if not + """ + latest_record_value: pendulum.DateTime = pendulum.parse(latest_record[self.cursor_field]) + slice_max_value: pendulum.DateTime = pendulum.parse(self._slice.get("end_date")) + max_possible_value = min(latest_record_value, slice_max_value) + if current_stream_state.get(self.cursor_field): + if latest_record_value > slice_max_value: + return {self.cursor_field: max_possible_value.isoformat()} + max_possible_value = max(latest_record_value, pendulum.parse(current_stream_state[self.cursor_field])) + return {self.cursor_field: max_possible_value.isoformat()} + + +class BulkIncrementalSalesforceStream(BulkSalesforceStream, IncrementalRestSalesforceStream): + state_checkpoint_interval = None + + def request_params( + self, stream_state: Mapping[str, Any], stream_slice: Mapping[str, Any] = None, next_page_token: Mapping[str, Any] = None + ) -> MutableMapping[str, Any]: + start_date = stream_slice["start_date"] + end_date = stream_slice["end_date"] + + select_fields = self.get_query_select_fields() + table_name = self.name + where_conditions = [f"{self.cursor_field} >= {start_date}", f"{self.cursor_field} < {end_date}"] + + where_clause = f"WHERE {' AND '.join(where_conditions)}" + query = f"SELECT {select_fields} FROM {table_name} {where_clause}" + return {"q": query} + + +class Describe(Stream): + """ + Stream of sObjects' (Salesforce Objects) describe: + https://developer.salesforce.com/docs/atlas.en-us.api_rest.meta/api_rest/resources_sobject_describe.htm + """ + + name = "Describe" + primary_key = "name" + + def __init__(self, sf_api: Salesforce, catalog: ConfiguredAirbyteCatalog = None, **kwargs): + super().__init__(**kwargs) + self.sf_api = sf_api + if catalog: + self.sobjects_to_describe = [s.stream.name for s in catalog.streams if s.stream.name != self.name] + + def read_records(self, **kwargs) -> Iterable[Mapping[str, Any]]: + """ + Yield describe response of SObjects defined in catalog as streams only. + """ + for sobject in self.sobjects_to_describe: + yield self.sf_api.describe(sobject=sobject) diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/utils.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/utils.py similarity index 100% rename from airbyte-integrations/connectors/source-salesforce/source_salesforce/async_salesforce/utils.py rename to airbyte-integrations/connectors/source-salesforce/source_salesforce/utils.py diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py index 21ccdd28402c..8f87e2bd58cd 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test.py @@ -1,7 +1,8 @@ # # Copyright (c) 2023 Airbyte, Inc., all rights reserved. # -import asyncio + + import csv import io import logging @@ -9,30 +10,29 @@ from datetime import datetime from typing import List from unittest.mock import Mock -from yarl import URL import freezegun import pendulum import pytest import requests_mock -from aioresponses import CallbackResult, aioresponses from airbyte_cdk.models import AirbyteStream, ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, DestinationSyncMode, SyncMode, Type -from airbyte_cdk.sources.async_cdk import source_dispatcher from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade -from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils import AirbyteTracedException from conftest import encoding_symbols_parameters, generate_stream -from source_salesforce.async_salesforce.api import Salesforce -from source_salesforce.async_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING -from source_salesforce.async_salesforce.source import SalesforceSourceDispatcher, AsyncSourceSalesforce -from source_salesforce.async_salesforce.streams import ( +from requests.exceptions import HTTPError +from source_salesforce.api import Salesforce +from source_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING +from source_salesforce.source import SourceSalesforce +from source_salesforce.streams import ( CSV_FIELD_SIZE_LIMIT, BulkIncrementalSalesforceStream, BulkSalesforceStream, BulkSalesforceSubStream, + Describe, IncrementalRestSalesforceStream, RestSalesforceStream, + SalesforceStream, ) _ANY_CATALOG = ConfiguredAirbyteCatalog.parse_obj({"streams": []}) @@ -65,7 +65,7 @@ def test_login_authentication_error_handler( stream_config, requests_mock, login_status_code, login_json_resp, expected_error_msg, is_config_error ): - source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) logger = logging.getLogger("airbyte") requests_mock.register_uri( "POST", "https://login.salesforce.com/services/oauth2/token", json=login_json_resp, status_code=login_status_code @@ -81,209 +81,156 @@ def test_login_authentication_error_handler( assert msg == expected_error_msg -@pytest.mark.asyncio -async def test_bulk_sync_creation_failed(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() - - def callback(*args, **kwargs): - return CallbackResult(status=400, payload={"message": "test_error"}) - - with aioresponses() as m: - m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) - with pytest.raises(HttpError) as err: - stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) - [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] - - assert err.value.json()["message"] == "test_error" - await stream._session.close() +def test_bulk_sync_creation_failed(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + with requests_mock.Mocker() as m: + m.register_uri("POST", stream.path(), status_code=400, json=[{"message": "test_error"}]) + with pytest.raises(HTTPError) as err: + stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) + next(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)) + assert err.value.response.json()[0]["message"] == "test_error" -@pytest.mark.asyncio -async def test_bulk_stream_fallback_to_rest(stream_config, stream_api): +def test_bulk_stream_fallback_to_rest(mocker, requests_mock, stream_config, stream_api): """ Here we mock BULK API with response returning error, saying BULK is not supported for this kind of entity. On the other hand, we mock REST API for this same entity with a successful response. After having instantiated a BulkStream, sync should succeed in case it falls back to REST API. Otherwise it would throw an error. """ - stream = await generate_stream("CustomEntity", stream_config, stream_api) - await stream.ensure_session() - - def callback(*args, **kwargs): - return CallbackResult(status=400, payload={"errorCode": "INVALIDENTITY", "message": "CustomEntity is not supported by the Bulk API"}, content_type="application/json") - + stream = generate_stream("CustomEntity", stream_config, stream_api) + # mock a BULK API + requests_mock.register_uri( + "POST", + "https://fase-account.salesforce.com/services/data/v57.0/jobs/query", + status_code=400, + json=[{"errorCode": "INVALIDENTITY", "message": "CustomEntity is not supported by the Bulk API"}], + ) rest_stream_records = [ {"id": 1, "name": "custom entity", "created": "2010-11-11"}, {"id": 11, "name": "custom entity", "created": "2020-01-02"}, ] - async def get_records(*args, **kwargs): - nonlocal rest_stream_records - for record in rest_stream_records: - yield record + # mock REST API + mocker.patch("source_salesforce.source.RestSalesforceStream.read_records", lambda *args, **kwargs: iter(rest_stream_records)) + assert type(stream) is BulkIncrementalSalesforceStream + stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) + assert list(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)) == rest_stream_records - with aioresponses() as m: - # mock a BULK API - m.post("https://fase-account.salesforce.com/services/data/v57.0/jobs/query", status=400, callback=callback) - # mock REST API - stream.read_records = get_records - assert type(stream) is BulkIncrementalSalesforceStream - stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) - assert [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] == rest_stream_records - await stream._session.close() - - -@pytest.mark.asyncio -async def test_stream_unsupported_by_bulk(stream_config, stream_api): +def test_stream_unsupported_by_bulk(stream_config, stream_api): """ Stream `AcceptedEventRelation` is not supported by BULK API, so that REST API stream will be used for it. """ stream_name = "AcceptedEventRelation" - stream = await generate_stream(stream_name, stream_config, stream_api) + stream = generate_stream(stream_name, stream_config, stream_api) assert not isinstance(stream, BulkSalesforceStream) -@pytest.mark.asyncio -async def test_stream_contains_unsupported_properties_by_bulk(stream_config, stream_api_v2): +def test_stream_contains_unsupported_properties_by_bulk(stream_config, stream_api_v2): """ Stream `Account` contains compound field such as BillingAddress, which is not supported by BULK API (csv), in that case REST API stream will be used for it. """ stream_name = "Account" - stream = await generate_stream(stream_name, stream_config, stream_api_v2) + stream = generate_stream(stream_name, stream_config, stream_api_v2) assert not isinstance(stream, BulkSalesforceStream) -@pytest.mark.asyncio -async def test_bulk_sync_pagination(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() +def test_bulk_sync_pagination(stream_config, stream_api, requests_mock): + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) job_id = "fake_job" - call_counter = 0 - - def cb1(*args, **kwargs): - nonlocal call_counter - call_counter += 1 - return CallbackResult(headers={"Sforce-Locator": "somelocator_1"}, body="\n".join(resp_text)) - - def cb2(*args, **kwargs): - nonlocal call_counter - call_counter += 1 - return CallbackResult(headers={"Sforce-Locator": "somelocator_2"}, body="\n".join(resp_text)) - - def cb3(*args, **kwargs): - nonlocal call_counter - call_counter += 1 - return CallbackResult(headers={"Sforce-Locator": "null"}, body="\n".join(resp_text)) + requests_mock.register_uri("POST", stream.path(), json={"id": job_id}) + requests_mock.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "JobComplete"}) + resp_text = ["Field1,LastModifiedDate,ID"] + [f"test,2021-11-16,{i}" for i in range(5)] + result_uri = requests_mock.register_uri( + "GET", + stream.path() + f"/{job_id}/results", + [ + {"text": "\n".join(resp_text), "headers": {"Sforce-Locator": "somelocator_1"}}, + {"text": "\n".join(resp_text), "headers": {"Sforce-Locator": "somelocator_2"}}, + {"text": "\n".join(resp_text), "headers": {"Sforce-Locator": "null"}}, + ], + ) + requests_mock.register_uri("DELETE", stream.path() + f"/{job_id}") - with aioresponses() as m: - base_url = f"{stream.sf_api.instance_url}{stream.path()}" - m.post(f"{base_url}", callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) - m.get(f"{base_url}/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) - resp_text = ["Field1,LastModifiedDate,ID"] + [f"test,2021-11-16,{i}" for i in range(5)] - m.get(f"{base_url}/{job_id}/results", callback=cb1) - m.get(f"{base_url}/{job_id}/results?locator=somelocator_1", callback=cb2) - m.get(f"{base_url}/{job_id}/results?locator=somelocator_2", callback=cb3) - m.delete(base_url + f"/{job_id}") + stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) + loaded_ids = [int(record["ID"]) for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + assert loaded_ids == [0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4] + assert result_uri.call_count == 3 + assert result_uri.request_history[1].query == "locator=somelocator_1" + assert result_uri.request_history[2].query == "locator=somelocator_2" - stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) - loaded_ids = [int(record["ID"]) async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] - assert loaded_ids == [0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4] - assert call_counter == 3 - await stream._session.close() def _prepare_mock(m, stream): job_id = "fake_job_1" - base_url = f"{stream.sf_api.instance_url}{stream.path()}" - m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) - m.delete(base_url + f"/{job_id}") - m.get(base_url + f"/{job_id}/results", callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2021-11-16,1")) - m.patch(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(body="")) + m.register_uri("POST", stream.path(), json={"id": job_id}) + m.register_uri("DELETE", stream.path() + f"/{job_id}") + m.register_uri("GET", stream.path() + f"/{job_id}/results", text="Field1,LastModifiedDate,ID\ntest,2021-11-16,1") + m.register_uri("PATCH", stream.path() + f"/{job_id}", text="") return job_id -async def _get_result_id(stream): - stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) - records = [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] - return int(list(records)[0]["ID"]) - - -@pytest.mark.asyncio -async def test_bulk_sync_successful(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() - base_url = f"{stream.sf_api.instance_url}{stream.path()}" +def _get_result_id(stream): + stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) + return int(list(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices))[0]["ID"]) - with aioresponses() as m: - m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) - with aioresponses() as m: +def test_bulk_sync_successful(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + with requests_mock.Mocker() as m: job_id = _prepare_mock(m, stream) - m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) - assert await _get_result_id(stream) == 1 - + m.register_uri("GET", stream.path() + f"/{job_id}", [{"json": {"state": "JobComplete"}}]) + assert _get_result_id(stream) == 1 -@pytest.mark.asyncio -async def test_bulk_sync_successful_long_response(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() - base_url = f"{stream.sf_api.instance_url}{stream.path()}" - with aioresponses() as m: +def test_bulk_sync_successful_long_response(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + with requests_mock.Mocker() as m: job_id = _prepare_mock(m, stream) - m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "UploadComplete", "id": job_id})) - m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) - m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete", "id": job_id})) - assert await _get_result_id(stream) == 1 + m.register_uri( + "GET", + stream.path() + f"/{job_id}", + [ + {"json": {"state": "UploadComplete", "id": job_id}}, + {"json": {"state": "InProgress", "id": job_id}}, + {"json": {"state": "JobComplete", "id": job_id}}, + ], + ) + assert _get_result_id(stream) == 1 # maximum timeout is wait_timeout * max_retry_attempt # this test tries to check a job state 17 times with +-1second for very one -@pytest.mark.asyncio @pytest.mark.timeout(17) -async def test_bulk_sync_successful_retry(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) +def test_bulk_sync_successful_retry(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds - await stream.ensure_session() - base_url = f"{stream.sf_api.instance_url}{stream.path()}" - with aioresponses() as m: + with requests_mock.Mocker() as m: job_id = _prepare_mock(m, stream) # 2 failed attempts, 3rd one should be successful states = [{"json": {"state": "InProgress", "id": job_id}}] * 17 states.append({"json": {"state": "JobComplete", "id": job_id}}) # raise Exception(states) - for _ in range(17): - m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) - m.get(base_url + f"/{job_id}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete", "id": job_id})) - - assert await _get_result_id(stream) == 1 + m.register_uri("GET", stream.path() + f"/{job_id}", states) + assert _get_result_id(stream) == 1 -@pytest.mark.asyncio @pytest.mark.timeout(30) -async def test_bulk_sync_failed_retry(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) +def test_bulk_sync_failed_retry(stream_config, stream_api): + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds - await stream.ensure_session() - base_url = f"{stream.sf_api.instance_url}{stream.path()}" - - with aioresponses() as m: + with requests_mock.Mocker() as m: job_id = _prepare_mock(m, stream) - m.get(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"state": "InProgress", "id": job_id})) - m.post(base_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) + m.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "InProgress", "id": job_id}) with pytest.raises(Exception) as err: - stream_slices = await anext(stream.stream_slices(sync_mode=SyncMode.incremental)) - [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)] + stream_slices = next(iter(stream.stream_slices(sync_mode=SyncMode.incremental))) + next(stream.read_records(sync_mode=SyncMode.full_refresh, stream_slice=stream_slices)) assert "stream using BULK API was failed" in str(err.value) - await stream._session.close() - -@pytest.mark.asyncio @pytest.mark.parametrize( "start_date_provided,stream_name,expected_start_date", [ @@ -291,7 +238,7 @@ async def test_bulk_sync_failed_retry(stream_config, stream_api): (True, "ActiveFeatureLicenseMetric", "2010-01-18T21:18:20Z"), ], ) -async def test_stream_start_date( +def test_stream_start_date( start_date_provided, stream_name, expected_start_date, @@ -300,96 +247,84 @@ async def test_stream_start_date( stream_config_without_start_date, ): if start_date_provided: - stream = await generate_stream(stream_name, stream_config, stream_api) + stream = generate_stream(stream_name, stream_config, stream_api) assert stream.start_date == expected_start_date else: - stream = await generate_stream(stream_name, stream_config_without_start_date, stream_api) + stream = generate_stream(stream_name, stream_config_without_start_date, stream_api) assert datetime.strptime(stream.start_date, "%Y-%m-%dT%H:%M:%SZ").year == datetime.now().year - 2 -@pytest.mark.asyncio -async def test_stream_start_date_should_be_converted_to_datetime_format(stream_config_date_format, stream_api): - stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) +def test_stream_start_date_should_be_converted_to_datetime_format(stream_config_date_format, stream_api): + stream: IncrementalRestSalesforceStream = generate_stream("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) assert stream.start_date == "2010-01-18T00:00:00Z" -@pytest.mark.asyncio -async def test_stream_start_datetime_format_should_not_changed(stream_config, stream_api): - stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config, stream_api) +def test_stream_start_datetime_format_should_not_changed(stream_config, stream_api): + stream: IncrementalRestSalesforceStream = generate_stream("ActiveFeatureLicenseMetric", stream_config, stream_api) assert stream.start_date == "2010-01-18T21:18:20Z" -@pytest.mark.asyncio -async def test_download_data_filter_null_bytes(stream_config, stream_api): +def test_download_data_filter_null_bytes(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with aioresponses() as m: - m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b"\x00")) - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + with requests_mock.Mocker() as m: + m.register_uri("GET", job_full_url_results, content=b"\x00") + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [] - m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b'"Id","IsDeleted"\n\x00"0014W000027f6UwQAI","false"\n\x00\x00')) - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + m.register_uri("GET", job_full_url_results, content=b'"Id","IsDeleted"\n\x00"0014W000027f6UwQAI","false"\n\x00\x00') + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"Id": "0014W000027f6UwQAI", "IsDeleted": "false"}] -@pytest.mark.asyncio -async def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): +def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with aioresponses() as m: - m.get(job_full_url_results, callback=lambda *args, **kwargs: CallbackResult(body=b'"IsDeleted","Age"\n"false",24\n')) - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + with requests_mock.Mocker() as m: + m.register_uri("GET", job_full_url_results, content=b'"IsDeleted","Age"\n"false",24\n') + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"IsDeleted": "false", "Age": "24"}] -@pytest.mark.asyncio -async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): +def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with aioresponses() as m: - m.get(job_full_url_results, body=b'"ZipCode"\n"01234"\n') - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + with requests_mock.Mocker() as m: + m.register_uri("GET", job_full_url_results, content=b'"ZipCode"\n"01234"\n') + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"ZipCode": "01234"}] -@pytest.mark.asyncio -async def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): +def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with aioresponses() as m: - m.get(job_full_url_results, body=b'"IsDeleted","Age","Name"\n"false",,"Airbyte"\n') - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + with requests_mock.Mocker() as m: + m.register_uri("GET", job_full_url_results, content=b'"IsDeleted","Age","Name"\n"false",,"Airbyte"\n') + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == [{"IsDeleted": "false", "Age": None, "Name": "Airbyte"}] -@pytest.mark.asyncio @pytest.mark.parametrize( "chunk_size, content_type_header, content, expected_result", encoding_symbols_parameters(), ids=[f"charset: {x[1]}, chunk_size: {x[0]}" for x in encoding_symbols_parameters()], ) -async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): +def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) - with aioresponses() as m: - m.get(job_full_url_results, headers=content_type_header, body=content) - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + with requests_mock.Mocker() as m: + m.register_uri("GET", job_full_url_results, headers=content_type_header, content=content) + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) res = list(stream.read_with_chunks(tmp_file, response_encoding)) assert res == expected_result @@ -407,10 +342,10 @@ async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_t ), ), ) -async def test_check_connection_rate_limit( +def test_check_connection_rate_limit( stream_config, login_status_code, login_json_resp, discovery_status_code, discovery_resp_json, expected_error_msg ): - source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) logger = logging.getLogger("airbyte") with requests_mock.Mocker() as m: @@ -438,81 +373,72 @@ def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): While reading `stream_1` if 403 (Rate Limit) is received, it should finish that stream with success and stop the sync process. Next streams should not be executed. """ - source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time stream_config.update({"start_date": "2021-10-01"}) - loop = asyncio.get_event_loop() - stream_1: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Account", stream_config, stream_api)) - stream_2: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Asset", stream_config, stream_api)) + stream_1: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) + stream_2: BulkIncrementalSalesforceStream = generate_stream("Asset", stream_config, stream_api) streams = [stream_1, stream_2] configure_request_params_mock(stream_1, stream_2) stream_1.page_size = 6 stream_1.state_checkpoint_interval = 5 - source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.streams = Mock() source.streams.return_value = streams logger = logging.getLogger("airbyte") - json_response = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} - - orig_read_stream = source.async_source.read_stream - - async def patched_read_stream(*args, **kwargs): - base_url = f"{stream_1.sf_api.instance_url}{stream_1.path()}" - with aioresponses() as m: + json_response = [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."}] + with requests_mock.Mocker() as m: + for stream in streams: creation_responses = [] for page in [1, 2]: - job_id = f"fake_job_{page}_{stream_1.name}" - creation_responses.append({"id": job_id}) + job_id = f"fake_job_{page}_{stream.name}" + creation_responses.append({"json": {"id": job_id}}) - m.get(base_url + f"/{job_id}", callback=lambda *_, **__: CallbackResult(payload={"state": "JobComplete"})) + m.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "JobComplete"}) resp = ["Field1,LastModifiedDate,Id"] + [f"test,2021-10-0{i},{i}" for i in range(1, 7)] # 6 records per page if page == 1: # Read the first page successfully - m.get(base_url + f"/{job_id}/results", callback=lambda *_, **__: CallbackResult(body="\n".join(resp))) + m.register_uri("GET", stream.path() + f"/{job_id}/results", text="\n".join(resp)) else: # Requesting for results when reading second page should fail with 403 (Rate Limit error) - m.get(base_url + f"/{job_id}/results", status=403, callback=lambda *_, **__: CallbackResult(status=403, payload=json_response)) + m.register_uri("GET", stream.path() + f"/{job_id}/results", status_code=403, json=json_response) - m.delete(base_url + f"/{job_id}") + m.register_uri("DELETE", stream.path() + f"/{job_id}") - def cb(response): - return lambda *_, **__: CallbackResult(payload=response) + m.register_uri("POST", stream.path(), creation_responses) - for response in creation_responses: - m.post(base_url, callback=cb(response)) + result = [i for i in source.read(logger=logger, config=stream_config, catalog=bulk_catalog, state=state)] + assert stream_1.request_params.called + assert ( + not stream_2.request_params.called + ), "The second stream should not be executed, because the first stream finished with Rate Limit." - async for r in orig_read_stream(**kwargs): - yield r + records = [item for item in result if item.type == Type.RECORD] + assert len(records) == 6 # stream page size: 6 - source.async_source.read_stream = patched_read_stream + state_record = [item for item in result if item.type == Type.STATE][0] + assert state_record.state.data["Account"]["LastModifiedDate"] == "2021-10-05T00:00:00+00:00" # state checkpoint interval is 5. - result = [i for i in source.read(logger=logger, config=stream_config, catalog=bulk_catalog, state=state)] - assert stream_1.request_params.called - assert ( - not stream_2.request_params.called - ), "The second stream should not be executed, because the first stream finished with Rate Limit." - - records = [item for item in result if item.type == Type.RECORD] - assert len(records) == 6 # stream page size: 6 - state_record = [item for item in result if item.type == Type.STATE][0] - assert state_record.state.data["Account"]["LastModifiedDate"] == "2021-10-05T00:00:00+00:00" # state checkpoint interval is 5. +def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): + """ + Connector should stop the sync if one stream reached rate limit + stream_1, stream_2, stream_3, ... + While reading `stream_1` if 403 (Rate Limit) is received, it should finish that stream with success and stop the sync process. + Next streams should not be executed. + """ + stream_config.update({"start_date": "2021-11-01"}) + stream_1: IncrementalRestSalesforceStream = generate_stream("KnowledgeArticle", stream_config, stream_api) + stream_2: IncrementalRestSalesforceStream = generate_stream("AcceptedEventRelation", stream_config, stream_api) -@pytest.mark.asyncio -async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): - source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time - stream_config.update({"start_date": "2021-11-01"}) - stream_1: IncrementalRestSalesforceStream = await generate_stream("KnowledgeArticle", stream_config, stream_api) - stream_2: IncrementalRestSalesforceStream = await generate_stream("AcceptedEventRelation", stream_config, stream_api) stream_1.state_checkpoint_interval = 3 configure_request_params_mock(stream_1, stream_2) - source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.streams = Mock() source.streams.return_value = [stream_1, stream_2] @@ -546,105 +472,72 @@ async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): }, ], } - response_2 = {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."} + response_2 = [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "TotalRequests Limit exceeded."}] - def cb1(*args, **kwargs): - return CallbackResult(payload=response_1, status=200) - - def cb2(*args, **kwargs): - return CallbackResult(payload=response_2, status=403, reason="") - - orig_read_records_s1 = stream_1.read_records - orig_read_records_s2 = stream_2.read_records - - async def patched_read_records_s1(*args, **kwargs): - with aioresponses() as m: - m.get(re.compile(re.escape(rf"{stream_1.sf_api.instance_url}{stream_1.path()}") + rf"\??.*"), repeat=True, callback=cb1) - m.get(re.compile(re.escape(rf"{stream_1.sf_api.instance_url}{next_page_url}") + rf"\??.*"), repeat=True, callback=cb2) - - async for r in orig_read_records_s1(**kwargs): - yield r - - async def patched_read_records_s2(*args, **kwargs): - with aioresponses() as m: - m.get(re.compile(re.escape(rf"{stream_2.sf_api.instance_url}{stream_2.path()}") + rf"\??.*"), repeat=True, callback=cb1) - m.get(re.compile(re.escape(rf"{stream_2.sf_api.instance_url}{next_page_url}") + rf"\??.*"), repeat=True, callback=cb1) - async for r in orig_read_records_s2(**kwargs): - yield r - - async def check_availability(*args, **kwargs): - return (True, None) - - stream_1.read_records = lambda *args, **kwargs: patched_read_records_s1(stream_1, *args, **kwargs) - stream_1.check_availability = check_availability - stream_2.read_records = lambda *args, **kwargs: patched_read_records_s2(stream_2, *args, **kwargs) - stream_2.check_availability = check_availability + with requests_mock.Mocker() as m: + m.register_uri("GET", stream_1.path(), json=response_1, status_code=200) + m.register_uri("GET", next_page_url, json=response_2, status_code=403) - result = [i for i in source.read(logger=logger, config=stream_config, catalog=rest_catalog, state=state)] + result = [i for i in source.read(logger=logger, config=stream_config, catalog=rest_catalog, state=state)] - assert stream_1.request_params.called - assert ( - not stream_2.request_params.called - ), "The second stream should not be executed, because the first stream finished with Rate Limit." + assert stream_1.request_params.called + assert ( + not stream_2.request_params.called + ), "The second stream should not be executed, because the first stream finished with Rate Limit." - records = [item for item in result if item.type == Type.RECORD] - assert len(records) == 5 + records = [item for item in result if item.type == Type.RECORD] + assert len(records) == 5 - state_record = [item for item in result if item.type == Type.STATE][0] - assert state_record.state.data["KnowledgeArticle"]["LastModifiedDate"] == "2021-11-17T00:00:00+00:00" + state_record = [item for item in result if item.type == Type.STATE][0] + assert state_record.state.data["KnowledgeArticle"]["LastModifiedDate"] == "2021-11-17T00:00:00+00:00" -@pytest.mark.asyncio -async def test_pagination_rest(stream_config, stream_api): +def test_pagination_rest(stream_config, stream_api): stream_name = "AcceptedEventRelation" - stream: RestSalesforceStream = await generate_stream(stream_name, stream_config, stream_api) + stream: RestSalesforceStream = generate_stream(stream_name, stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds next_page_url = "/services/data/v57.0/query/012345" - await stream.ensure_session() - - resp_1 = { - "done": False, - "totalSize": 4, - "nextRecordsUrl": next_page_url, - "records": [ - { - "ID": 1, - "LastModifiedDate": "2021-11-15", - }, - { - "ID": 2, - "LastModifiedDate": "2021-11-16", - }, - ], - } - resp_2 = { - "done": True, - "totalSize": 4, - "records": [ - { - "ID": 3, - "LastModifiedDate": "2021-11-17", - }, - { - "ID": 4, - "LastModifiedDate": "2021-11-18", - }, - ], - } - - with aioresponses() as m: - m.get(re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0\??.*"), callback=lambda *args, **kwargs: CallbackResult(payload=resp_1)) - m.get("https://fase-account.salesforce.com" + next_page_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload=resp_2)) - - records = [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh)] + with requests_mock.Mocker() as m: + resp_1 = { + "done": False, + "totalSize": 4, + "nextRecordsUrl": next_page_url, + "records": [ + { + "ID": 1, + "LastModifiedDate": "2021-11-15", + }, + { + "ID": 2, + "LastModifiedDate": "2021-11-16", + }, + ], + } + resp_2 = { + "done": True, + "totalSize": 4, + "records": [ + { + "ID": 3, + "LastModifiedDate": "2021-11-17", + }, + { + "ID": 4, + "LastModifiedDate": "2021-11-18", + }, + ], + } + + m.register_uri("GET", stream.path(), json=resp_1) + m.register_uri("GET", next_page_url, json=resp_2) + + records = [record for record in stream.read_records(sync_mode=SyncMode.full_refresh)] assert len(records) == 4 -@pytest.mark.asyncio -async def test_csv_reader_dialect_unix(): +def test_csv_reader_dialect_unix(): stream: BulkSalesforceStream = BulkSalesforceStream(stream_name=None, sf_api=None, pk=None) url_results = "https://fake-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - await stream.ensure_session() data = [ {"Id": "1", "Name": '"first_name" "last_name"'}, @@ -659,9 +552,9 @@ async def test_csv_reader_dialect_unix(): writer.writerow(line) text = csvfile.getvalue() - with aioresponses() as m: - m.get(url_results, callback=lambda *args, **kwargs: CallbackResult(body=text)) - tmp_file, response_encoding, _ = await stream.download_data(url=url_results) + with requests_mock.Mocker() as m: + m.register_uri("GET", url_results, text=text) + tmp_file, response_encoding, _ = stream.download_data(url=url_results) result = [i for i in stream.read_with_chunks(tmp_file, response_encoding)] assert result == data @@ -683,7 +576,7 @@ async def test_csv_reader_dialect_unix(): ), ), ) -async def test_forwarding_sobject_options(stream_config, stream_names, catalog_stream_names) -> None: +def test_forwarding_sobject_options(stream_config, stream_names, catalog_stream_names) -> None: sobjects_matcher = re.compile("/sobjects$") token_matcher = re.compile("/token$") describe_matcher = re.compile("/describe$") @@ -730,7 +623,7 @@ async def test_forwarding_sobject_options(stream_config, stream_names, catalog_s ], }, ) - source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.catalog = catalog streams = source.streams(config=stream_config) expected_names = catalog_stream_names if catalog else stream_names @@ -745,6 +638,46 @@ async def test_forwarding_sobject_options(stream_config, stream_names, catalog_s return +@pytest.mark.parametrize( + "stream_names,catalog_stream_names,", + ( + ( + ["stream_1", "stream_2", "Describe"], + None, + ), + ( + ["stream_1", "stream_2"], + ["stream_1", "stream_2", "Describe"], + ), + ( + ["stream_1", "stream_2", "stream_3", "Describe"], + ["stream_1", "Describe"], + ), + ), +) +def test_unspecified_and_incremental_streams_are_not_concurrent(stream_config, stream_names, catalog_stream_names) -> None: + for stream in _get_streams(stream_config, stream_names, catalog_stream_names, SyncMode.incremental): + assert isinstance(stream, (SalesforceStream, Describe)) + + +@pytest.mark.parametrize( + "stream_names,catalog_stream_names,", + ( + ( + ["stream_1", "stream_2"], + ["stream_1", "stream_2", "Describe"], + ), + ( + ["stream_1", "stream_2", "stream_3", "Describe"], + ["stream_1", "Describe"], + ), + ), +) +def test_full_refresh_streams_are_concurrent(stream_config, stream_names, catalog_stream_names) -> None: + for stream in _get_streams(stream_config, stream_names, catalog_stream_names, SyncMode.full_refresh): + assert isinstance(stream, StreamFacade) + + def _get_streams(stream_config, stream_names, catalog_stream_names, sync_type) -> List[Stream]: sobjects_matcher = re.compile("/sobjects$") token_matcher = re.compile("/token$") @@ -790,7 +723,7 @@ def _get_streams(stream_config, stream_names, catalog_stream_names, sync_type) - ], }, ) - source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.catalog = catalog return source.streams(config=stream_config) @@ -813,25 +746,21 @@ def test_csv_field_size_limit(): pass -@pytest.mark.asyncio -async def test_convert_to_standard_instance(stream_config, stream_api): - bulk_stream = await generate_stream("Account", stream_config, stream_api) +def test_convert_to_standard_instance(stream_config, stream_api): + bulk_stream = generate_stream("Account", stream_config, stream_api) rest_stream = bulk_stream.get_standard_instance() assert isinstance(rest_stream, IncrementalRestSalesforceStream) -@pytest.mark.asyncio -async def test_rest_stream_init_with_too_many_properties(stream_config, stream_api_v2_too_many_properties): +def test_rest_stream_init_with_too_many_properties(stream_config, stream_api_v2_too_many_properties): with pytest.raises(AssertionError): # v2 means the stream is going to be a REST stream. # A missing primary key is not allowed - await generate_stream("Account", stream_config, stream_api_v2_too_many_properties) + generate_stream("Account", stream_config, stream_api_v2_too_many_properties) -@pytest.mark.asyncio -async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): - stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) - await stream.ensure_session() +def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): + stream = generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) chunks = list(stream.chunk_properties()) for chunk in chunks: assert stream.primary_key in chunk @@ -839,27 +768,29 @@ async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_prop assert stream.too_many_properties assert stream.primary_key assert type(stream) == RestSalesforceStream - next_page_url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" - url_pattern = re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0/queryAll\??.*") - with aioresponses() as m: - m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={ + url = next_page_url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" + requests_mock.get( + url, + [ + { + "json": { "records": [ {"Id": 1, "propertyA": "A"}, {"Id": 2, "propertyA": "A"}, {"Id": 3, "propertyA": "A"}, {"Id": 4, "propertyA": "A"}, ] - })) - m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"nextRecordsUrl": next_page_url, "records": [{"Id": 1, "propertyB": "B"}, {"Id": 2, "propertyB": "B"}]})) - # 2 for 2 chunks above - for _ in range(chunks_len - 2): - m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 1}, {"Id": 2}], "nextRecordsUrl": next_page_url})) - m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 3, "propertyB": "B"}, {"Id": 4, "propertyB": "B"}]})) - # 2 for 1 chunk above and 1 chunk had no next page - for _ in range(chunks_len - 2): - m.get(url_pattern, callback=lambda *args, **kwargs: CallbackResult(payload={"records": [{"Id": 3}, {"Id": 4}]})) - - records = [r async for r in stream.read_records(sync_mode=SyncMode.full_refresh)] + } + }, + {"json": {"nextRecordsUrl": next_page_url, "records": [{"Id": 1, "propertyB": "B"}, {"Id": 2, "propertyB": "B"}]}}, + # 2 for 2 chunks above + *[{"json": {"records": [{"Id": 1}, {"Id": 2}], "nextRecordsUrl": next_page_url}} for _ in range(chunks_len - 2)], + {"json": {"records": [{"Id": 3, "propertyB": "B"}, {"Id": 4, "propertyB": "B"}]}}, + # 2 for 1 chunk above and 1 chunk had no next page + *[{"json": {"records": [{"Id": 3}, {"Id": 4}]}} for _ in range(chunks_len - 2)], + ], + ) + records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) assert records == [ {"Id": 1, "propertyA": "A", "propertyB": "B"}, {"Id": 2, "propertyA": "A", "propertyB": "B"}, @@ -870,89 +801,94 @@ async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_prop assert len(call.url) < Salesforce.REQUEST_SIZE_LIMITS -@pytest.mark.asyncio -async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties): - stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) +def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): + stream = generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) chunks = list(stream.chunk_properties()) for chunk in chunks: assert stream.primary_key in chunk assert stream.too_many_properties assert stream.primary_key assert type(stream) == RestSalesforceStream - url = re.compile(r"https://fase-account\.salesforce\.com/services/data/v57\.0/queryAll\??.*") - await stream.ensure_session() - - with aioresponses() as m: - m.get(url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"records": []})) - records = [record async for record in stream.read_records(sync_mode=SyncMode.full_refresh)] - assert records == [] + url = "https://fase-account.salesforce.com/services/data/v57.0/queryAll" + requests_mock.get( + url, + [ + {"json": {"records": []}}, + ], + ) + records = list(stream.read_records(sync_mode=SyncMode.full_refresh)) + assert records == [] -@pytest.mark.asyncio @pytest.mark.parametrize( "status_code,response_json,log_message", [ ( - 400, - {"errorCode": "INVALIDENTITY", "message": "Account is not supported by the Bulk API"}, - "Account is not supported by the Bulk API", + 400, + [{"errorCode": "INVALIDENTITY", "message": "Account is not supported by the Bulk API"}], + "Account is not supported by the Bulk API", ), - (403, {"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "API limit reached"}, "API limit reached"), - (400, {"errorCode": "API_ERROR", "message": "API does not support query"}, "The stream 'Account' is not queryable,"), + (403, [{"errorCode": "REQUEST_LIMIT_EXCEEDED", "message": "API limit reached"}], "API limit reached"), + (400, [{"errorCode": "API_ERROR", "message": "API does not support query"}], "The stream 'Account' is not queryable,"), ( - 400, - {"errorCode": "LIMIT_EXCEEDED", "message": "Max bulk v2 query jobs (10000) per 24 hrs has been reached (10021)"}, - "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed.", + 400, + [{"errorCode": "LIMIT_EXCEEDED", "message": "Max bulk v2 query jobs (10000) per 24 hrs has been reached (10021)"}], + "Your API key for Salesforce has reached its limit for the 24-hour period. We will resume replication once the limit has elapsed.", ), ], ) -async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api, status_code, response_json, log_message, caplog): - stream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() +def test_bulk_stream_error_in_logs_on_create_job(requests_mock, stream_config, stream_api, status_code, response_json, log_message, caplog): + """ """ + stream = generate_stream("Account", stream_config, stream_api) url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query" - - with aioresponses() as m: - m.post(url, status=status_code, callback=lambda *args, **kwargs: CallbackResult(status=status_code, payload=response_json, reason="")) - query = "Select Id, Subject from Account" - with caplog.at_level(logging.ERROR): - assert await stream.create_stream_job(query, url) is None, "this stream should be skipped" + requests_mock.register_uri( + "POST", + url, + status_code=status_code, + json=response_json, + ) + query = "Select Id, Subject from Account" + with caplog.at_level(logging.ERROR): + assert stream.create_stream_job(query, url) is None, "this stream should be skipped" # check logs assert log_message in caplog.records[-1].message -@pytest.mark.asyncio @pytest.mark.parametrize( "status_code,response_json,error_message", [ ( 400, - { - "errorCode": "TXN_SECURITY_METERING_ERROR", - "message": "We can't complete the action because enabled transaction security policies took too long to complete.", - }, + [ + { + "errorCode": "TXN_SECURITY_METERING_ERROR", + "message": "We can't complete the action because enabled transaction security policies took too long to complete.", + } + ], 'A transient authentication error occurred. To prevent future syncs from failing, assign the "Exempt from Transaction Security" user permission to the authenticated user.', ), ], ) -async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, status_code, response_json, error_message): - stream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() - url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query/queryJobId" - - with aioresponses() as m: - m.get(url, status=status_code, callback=lambda *args, **kwargs: CallbackResult(status=status_code, payload=response_json, reason="")) - with pytest.raises(AirbyteTracedException) as e: - await stream.wait_for_job(url=url) - assert e.value.message == error_message +def test_bulk_stream_error_on_wait_for_job(requests_mock, stream_config, stream_api, status_code, response_json, error_message): + stream = generate_stream("Account", stream_config, stream_api) + url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query/queryJobId" + requests_mock.register_uri( + "GET", + url, + status_code=status_code, + json=response_json, + ) + with pytest.raises(AirbyteTracedException) as e: + stream.wait_for_job(url=url) + assert e.value.message == error_message -@pytest.mark.asyncio() @freezegun.freeze_time("2023-01-01") -async def test_bulk_stream_slices(stream_config_date_format, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("FakeBulkStream", stream_config_date_format, stream_api) - stream_slices = [s async for s in stream.stream_slices(sync_mode=SyncMode.full_refresh)] +def test_bulk_stream_slices(stream_config_date_format, stream_api): + stream: BulkIncrementalSalesforceStream = generate_stream("FakeBulkStream", stream_config_date_format, stream_api) + stream_slices = list(stream.stream_slices(sync_mode=SyncMode.full_refresh)) expected_slices = [] today = pendulum.today(tz="UTC") start_date = pendulum.parse(stream.start_date, tz="UTC") @@ -966,108 +902,103 @@ async def test_bulk_stream_slices(stream_config_date_format, stream_api): start_date = start_date.add(days=stream.STREAM_SLICE_STEP) assert expected_slices == stream_slices - -@pytest.mark.asyncio @freezegun.freeze_time("2023-04-01") -async def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog): +def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog, requests_mock): + """Check that request params ignore records cursor and use start date from slice ONLY""" stream_config_date_format.update({"start_date": "2023-01-01"}) - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config_date_format, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config_date_format, stream_api) - source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.streams = Mock() source.streams.return_value = [stream] - base_url = f"{stream.sf_api.instance_url}{stream.path()}" job_id_1 = "fake_job_1" + requests_mock.register_uri("GET", stream.path() + f"/{job_id_1}", [{"json": {"state": "JobComplete"}}]) + requests_mock.register_uri("DELETE", stream.path() + f"/{job_id_1}") + requests_mock.register_uri("GET", stream.path() + f"/{job_id_1}/results", text="Field1,LastModifiedDate,ID\ntest,2023-01-15,1") + requests_mock.register_uri("PATCH", stream.path() + f"/{job_id_1}") + job_id_2 = "fake_job_2" + requests_mock.register_uri("GET", stream.path() + f"/{job_id_2}", [{"json": {"state": "JobComplete"}}]) + requests_mock.register_uri("DELETE", stream.path() + f"/{job_id_2}") + requests_mock.register_uri( + "GET", stream.path() + f"/{job_id_2}/results", text="Field1,LastModifiedDate,ID\ntest,2023-04-01,2\ntest,2023-02-20,22" + ) + requests_mock.register_uri("PATCH", stream.path() + f"/{job_id_2}") + job_id_3 = "fake_job_3" + queries_history = requests_mock.register_uri( + "POST", stream.path(), [{"json": {"id": job_id_1}}, {"json": {"id": job_id_2}}, {"json": {"id": job_id_3}}] + ) + requests_mock.register_uri("GET", stream.path() + f"/{job_id_3}", [{"json": {"state": "JobComplete"}}]) + requests_mock.register_uri("DELETE", stream.path() + f"/{job_id_3}") + requests_mock.register_uri("GET", stream.path() + f"/{job_id_3}/results", text="Field1,LastModifiedDate,ID\ntest,2023-04-01,3") + requests_mock.register_uri("PATCH", stream.path() + f"/{job_id_3}") - with aioresponses() as m: - m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_1})) - m.get(base_url + f"/{job_id_1}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) - m.delete(base_url + f"/{job_id_1}") - m.get(base_url + f"/{job_id_1}/results", - callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-01-15,1")) - m.patch(base_url + f"/{job_id_1}") - - m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_2})) - m.get(base_url + f"/{job_id_2}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) - m.delete(base_url + f"/{job_id_2}") - m.get(base_url + f"/{job_id_2}/results", - callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-04-01,2\ntest,2023-02-20,22")) - m.patch(base_url + f"/{job_id_2}") - - m.post(base_url, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id_3})) - m.get(base_url + f"/{job_id_3}", callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) - m.delete(base_url + f"/{job_id_3}") - m.get(base_url + f"/{job_id_3}/results", - callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2023-04-01,3")) - m.patch(base_url + f"/{job_id_3}") - - logger = logging.getLogger("airbyte") - state = {"Account": {"LastModifiedDate": "2023-01-01T10:10:10.000Z"}} - bulk_catalog.streams.pop(1) - result = [i for i in source.read(logger=logger, config=stream_config_date_format, catalog=bulk_catalog, state=state)] + logger = logging.getLogger("airbyte") + state = {"Account": {"LastModifiedDate": "2023-01-01T10:10:10.000Z"}} + bulk_catalog.streams.pop(1) + result = [i for i in source.read(logger=logger, config=stream_config_date_format, catalog=bulk_catalog, state=state)] actual_state_values = [item.state.data.get("Account").get(stream.cursor_field) for item in result if item.type == Type.STATE] - queries_history = m.requests - # assert request params assert ( - "LastModifiedDate >= 2023-01-01T10:10:10.000+00:00 AND LastModifiedDate < 2023-01-31T10:10:10.000+00:00" - in queries_history[("POST", URL(base_url))][0].kwargs["json"]["query"] + "LastModifiedDate >= 2023-01-01T10:10:10.000+00:00 AND LastModifiedDate < 2023-01-31T10:10:10.000+00:00" + in queries_history.request_history[0].text ) assert ( - "LastModifiedDate >= 2023-01-31T10:10:10.000+00:00 AND LastModifiedDate < 2023-03-02T10:10:10.000+00:00" - in queries_history[("POST", URL(base_url))][1].kwargs["json"]["query"] + "LastModifiedDate >= 2023-01-31T10:10:10.000+00:00 AND LastModifiedDate < 2023-03-02T10:10:10.000+00:00" + in queries_history.request_history[1].text ) assert ( - "LastModifiedDate >= 2023-03-02T10:10:10.000+00:00 AND LastModifiedDate < 2023-04-01T00:00:00.000+00:00" - in queries_history[("POST", URL(base_url))][2].kwargs["json"]["query"] + "LastModifiedDate >= 2023-03-02T10:10:10.000+00:00 AND LastModifiedDate < 2023-04-01T00:00:00.000+00:00" + in queries_history.request_history[2].text ) # assert states + # if connector meets record with cursor `2023-04-01` out of current slice range 2023-01-31 <> 2023-03-02, we ignore all other values and set state to slice end_date expected_state_values = ["2023-01-15T00:00:00+00:00", "2023-03-02T10:10:10+00:00", "2023-04-01T00:00:00+00:00"] assert actual_state_values == expected_state_values -@pytest.mark.asyncio -async def test_request_params_incremental(stream_config_date_format, stream_api): - stream = await generate_stream("ContentDocument", stream_config_date_format, stream_api) +def test_request_params_incremental(stream_config_date_format, stream_api): + stream = generate_stream("ContentDocument", stream_config_date_format, stream_api) params = stream.request_params(stream_state={}, stream_slice={'start_date': '2020', 'end_date': '2021'}) assert params == {'q': 'SELECT LastModifiedDate, Id FROM ContentDocument WHERE LastModifiedDate >= 2020 AND LastModifiedDate < 2021'} -@pytest.mark.asyncio -async def test_request_params_substream(stream_config_date_format, stream_api): - stream = await generate_stream("ContentDocumentLink", stream_config_date_format, stream_api) +def test_request_params_substream(stream_config_date_format, stream_api): + stream = generate_stream("ContentDocumentLink", stream_config_date_format, stream_api) params = stream.request_params(stream_state={}, stream_slice={'parents': [{'Id': 1}, {'Id': 2}]}) assert params == {"q": "SELECT LastModifiedDate, Id FROM ContentDocumentLink WHERE ContentDocumentId IN ('1','2')"} -@pytest.mark.asyncio @freezegun.freeze_time("2023-03-20") -async def test_stream_slices_for_substream(stream_config, stream_api): +def test_stream_slices_for_substream(stream_config, stream_api, requests_mock): + """Test BulkSalesforceSubStream for ContentDocumentLink (+ parent ContentDocument) + + ContentDocument return 1 record for each slice request. + Given start/end date leads to 3 date slice for ContentDocument, thus 3 total records + + ContentDocumentLink + It means that ContentDocumentLink should have 2 slices, with 2 and 1 records in each + """ stream_config['start_date'] = '2023-01-01' - stream: BulkSalesforceSubStream = await generate_stream("ContentDocumentLink", stream_config, stream_api) + stream: BulkSalesforceSubStream = generate_stream("ContentDocumentLink", stream_config, stream_api) stream.SLICE_BATCH_SIZE = 2 # each ContentDocumentLink should contain 2 records from parent ContentDocument stream - await stream.ensure_session() job_id = "fake_job" - base_url = f"{stream.sf_api.instance_url}{stream.path()}" - - with aioresponses() as m: - m.post(base_url, repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"id": job_id})) - m.get(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult(payload={"state": "JobComplete"})) - m.get(base_url + f"/{job_id}/results", repeat=True, callback=lambda *args, **kwargs: CallbackResult(body="Field1,LastModifiedDate,ID\ntest,2021-11-16,123", headers={"Sforce-Locator": "null"})) - m.delete(base_url + f"/{job_id}", repeat=True, callback=lambda *args, **kwargs: CallbackResult()) - - stream_slices = [slice async for slice in stream.stream_slices(sync_mode=SyncMode.full_refresh)] - assert stream_slices == [ - {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}, - {'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]}, - {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]} - ] + requests_mock.register_uri("POST", stream.path(), json={"id": job_id}) + requests_mock.register_uri("GET", stream.path() + f"/{job_id}", json={"state": "JobComplete"}) + requests_mock.register_uri("GET", stream.path() + f"/{job_id}/results", [{"text": "Field1,LastModifiedDate,ID\ntest,2021-11-16,123", "headers": {"Sforce-Locator": "null"}}]) + requests_mock.register_uri("DELETE", stream.path() + f"/{job_id}") + + stream_slices = list(stream.stream_slices(sync_mode=SyncMode.full_refresh)) + assert stream_slices == [ + {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}, + {'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]}, + {'parents': [{'Field1': 'test', 'ID': '123', 'LastModifiedDate': '2021-11-16'}]} + ] + diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py index afd9cbb8bbb1..19176a71115b 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/api_test_async.py @@ -22,11 +22,11 @@ from airbyte_cdk.sources.streams.concurrent.adapters import StreamFacade from airbyte_cdk.sources.streams.http.utils import HttpError from airbyte_cdk.utils import AirbyteTracedException -from conftest import encoding_symbols_parameters, generate_stream +from conftest import encoding_symbols_parameters, generate_stream_async from source_salesforce.api import Salesforce from source_salesforce.exceptions import AUTHENTICATION_ERROR_MESSAGE_MAPPING -from source_salesforce.async_salesforce.source import SalesforceSourceDispatcher, SourceSalesforce -from source_salesforce.streams import ( +from source_salesforce.async_salesforce.source import SalesforceSourceDispatcher, AsyncSourceSalesforce +from source_salesforce.async_salesforce.streams import ( CSV_FIELD_SIZE_LIMIT, BulkIncrementalSalesforceStream, BulkSalesforceStream, @@ -65,7 +65,7 @@ def test_login_authentication_error_handler( stream_config, requests_mock, login_status_code, login_json_resp, expected_error_msg, is_config_error ): - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) logger = logging.getLogger("airbyte") requests_mock.register_uri( "POST", "https://login.salesforce.com/services/oauth2/token", json=login_json_resp, status_code=login_status_code @@ -83,7 +83,7 @@ def test_login_authentication_error_handler( @pytest.mark.asyncio async def test_bulk_sync_creation_failed(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() def callback(*args, **kwargs): @@ -106,7 +106,7 @@ async def test_bulk_stream_fallback_to_rest(stream_config, stream_api): On the other hand, we mock REST API for this same entity with a successful response. After having instantiated a BulkStream, sync should succeed in case it falls back to REST API. Otherwise it would throw an error. """ - stream = await generate_stream("CustomEntity", stream_config, stream_api) + stream = await generate_stream_async("CustomEntity", stream_config, stream_api) await stream.ensure_session() def callback(*args, **kwargs): @@ -139,7 +139,7 @@ async def test_stream_unsupported_by_bulk(stream_config, stream_api): Stream `AcceptedEventRelation` is not supported by BULK API, so that REST API stream will be used for it. """ stream_name = "AcceptedEventRelation" - stream = await generate_stream(stream_name, stream_config, stream_api) + stream = await generate_stream_async(stream_name, stream_config, stream_api) assert not isinstance(stream, BulkSalesforceStream) @@ -150,13 +150,13 @@ async def test_stream_contains_unsupported_properties_by_bulk(stream_config, str in that case REST API stream will be used for it. """ stream_name = "Account" - stream = await generate_stream(stream_name, stream_config, stream_api_v2) + stream = await generate_stream_async(stream_name, stream_config, stream_api_v2) assert not isinstance(stream, BulkSalesforceStream) @pytest.mark.asyncio async def test_bulk_sync_pagination(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() job_id = "fake_job" call_counter = 0 @@ -213,7 +213,7 @@ async def _get_result_id(stream): @pytest.mark.asyncio async def test_bulk_sync_successful(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -228,7 +228,7 @@ async def test_bulk_sync_successful(stream_config, stream_api): @pytest.mark.asyncio async def test_bulk_sync_successful_long_response(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -245,7 +245,7 @@ async def test_bulk_sync_successful_long_response(stream_config, stream_api): @pytest.mark.asyncio @pytest.mark.timeout(17) async def test_bulk_sync_successful_retry(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -266,7 +266,7 @@ async def test_bulk_sync_successful_retry(stream_config, stream_api): @pytest.mark.asyncio @pytest.mark.timeout(30) async def test_bulk_sync_failed_retry(stream_config, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds await stream.ensure_session() base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -300,29 +300,29 @@ async def test_stream_start_date( stream_config_without_start_date, ): if start_date_provided: - stream = await generate_stream(stream_name, stream_config, stream_api) + stream = await generate_stream_async(stream_name, stream_config, stream_api) assert stream.start_date == expected_start_date else: - stream = await generate_stream(stream_name, stream_config_without_start_date, stream_api) + stream = await generate_stream_async(stream_name, stream_config_without_start_date, stream_api) assert datetime.strptime(stream.start_date, "%Y-%m-%dT%H:%M:%SZ").year == datetime.now().year - 2 @pytest.mark.asyncio async def test_stream_start_date_should_be_converted_to_datetime_format(stream_config_date_format, stream_api): - stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) + stream: IncrementalRestSalesforceStream = await generate_stream_async("ActiveFeatureLicenseMetric", stream_config_date_format, stream_api) assert stream.start_date == "2010-01-18T00:00:00Z" @pytest.mark.asyncio async def test_stream_start_datetime_format_should_not_changed(stream_config, stream_api): - stream: IncrementalRestSalesforceStream = await generate_stream("ActiveFeatureLicenseMetric", stream_config, stream_api) + stream: IncrementalRestSalesforceStream = await generate_stream_async("ActiveFeatureLicenseMetric", stream_config, stream_api) assert stream.start_date == "2010-01-18T21:18:20Z" @pytest.mark.asyncio async def test_download_data_filter_null_bytes(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -340,7 +340,7 @@ async def test_download_data_filter_null_bytes(stream_config, stream_api): @pytest.mark.asyncio async def test_read_with_chunks_should_return_only_object_data_type(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -353,7 +353,7 @@ async def test_read_with_chunks_should_return_only_object_data_type(stream_confi @pytest.mark.asyncio async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_digits_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -366,7 +366,7 @@ async def test_read_with_chunks_should_return_a_string_when_a_string_with_only_d @pytest.mark.asyncio async def test_read_with_chunks_should_return_null_value_when_no_data_is_provided(stream_config, stream_api): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -384,7 +384,7 @@ async def test_read_with_chunks_should_return_null_value_when_no_data_is_provide ) async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_type_header, content, expected_result): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() with aioresponses() as m: @@ -410,7 +410,7 @@ async def test_encoding_symbols(stream_config, stream_api, chunk_size, content_t async def test_check_connection_rate_limit( stream_config, login_status_code, login_json_resp, discovery_status_code, discovery_resp_json, expected_error_msg ): - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) logger = logging.getLogger("airbyte") with requests_mock.Mocker() as m: @@ -441,15 +441,15 @@ def test_rate_limit_bulk(stream_config, stream_api, bulk_catalog, state): source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time stream_config.update({"start_date": "2021-10-01"}) loop = asyncio.get_event_loop() - stream_1: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Account", stream_config, stream_api)) - stream_2: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream("Asset", stream_config, stream_api)) + stream_1: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream_async("Account", stream_config, stream_api)) + stream_2: BulkIncrementalSalesforceStream = loop.run_until_complete(generate_stream_async("Asset", stream_config, stream_api)) streams = [stream_1, stream_2] configure_request_params_mock(stream_1, stream_2) stream_1.page_size = 6 stream_1.state_checkpoint_interval = 5 - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = streams logger = logging.getLogger("airbyte") @@ -507,12 +507,12 @@ def cb(response): async def test_rate_limit_rest(stream_config, stream_api, rest_catalog, state): source_dispatcher.DEFAULT_SESSION_LIMIT = 1 # ensure that only one stream runs at a time stream_config.update({"start_date": "2021-11-01"}) - stream_1: IncrementalRestSalesforceStream = await generate_stream("KnowledgeArticle", stream_config, stream_api) - stream_2: IncrementalRestSalesforceStream = await generate_stream("AcceptedEventRelation", stream_config, stream_api) + stream_1: IncrementalRestSalesforceStream = await generate_stream_async("KnowledgeArticle", stream_config, stream_api) + stream_2: IncrementalRestSalesforceStream = await generate_stream_async("AcceptedEventRelation", stream_config, stream_api) stream_1.state_checkpoint_interval = 3 configure_request_params_mock(stream_1, stream_2) - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = [stream_1, stream_2] @@ -597,7 +597,7 @@ async def check_availability(*args, **kwargs): @pytest.mark.asyncio async def test_pagination_rest(stream_config, stream_api): stream_name = "AcceptedEventRelation" - stream: RestSalesforceStream = await generate_stream(stream_name, stream_config, stream_api) + stream: RestSalesforceStream = await generate_stream_async(stream_name, stream_config, stream_api) stream.DEFAULT_WAIT_TIMEOUT_SECONDS = 6 # maximum wait timeout will be 6 seconds next_page_url = "/services/data/v57.0/query/012345" await stream.ensure_session() @@ -730,7 +730,7 @@ async def test_forwarding_sobject_options(stream_config, stream_names, catalog_s ], }, ) - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.catalog = catalog streams = source.streams(config=stream_config) expected_names = catalog_stream_names if catalog else stream_names @@ -790,7 +790,7 @@ def _get_streams(stream_config, stream_names, catalog_stream_names, sync_type) - ], }, ) - source = SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) + source = AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG) source.catalog = catalog return source.streams(config=stream_config) @@ -815,7 +815,7 @@ def test_csv_field_size_limit(): @pytest.mark.asyncio async def test_convert_to_standard_instance(stream_config, stream_api): - bulk_stream = await generate_stream("Account", stream_config, stream_api) + bulk_stream = await generate_stream_async("Account", stream_config, stream_api) rest_stream = bulk_stream.get_standard_instance() assert isinstance(rest_stream, IncrementalRestSalesforceStream) @@ -825,12 +825,12 @@ async def test_rest_stream_init_with_too_many_properties(stream_config, stream_a with pytest.raises(AssertionError): # v2 means the stream is going to be a REST stream. # A missing primary key is not allowed - await generate_stream("Account", stream_config, stream_api_v2_too_many_properties) + await generate_stream_async("Account", stream_config, stream_api_v2_too_many_properties) @pytest.mark.asyncio async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_properties, requests_mock): - stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + stream = await generate_stream_async("Account", stream_config, stream_api_v2_pk_too_many_properties) await stream.ensure_session() chunks = list(stream.chunk_properties()) for chunk in chunks: @@ -872,7 +872,7 @@ async def test_too_many_properties(stream_config, stream_api_v2_pk_too_many_prop @pytest.mark.asyncio async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_pk_too_many_properties): - stream = await generate_stream("Account", stream_config, stream_api_v2_pk_too_many_properties) + stream = await generate_stream_async("Account", stream_config, stream_api_v2_pk_too_many_properties) chunks = list(stream.chunk_properties()) for chunk in chunks: assert stream.primary_key in chunk @@ -907,7 +907,7 @@ async def test_stream_with_no_records_in_response(stream_config, stream_api_v2_p ], ) async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api, status_code, response_json, log_message, caplog): - stream = await generate_stream("Account", stream_config, stream_api) + stream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query" @@ -936,7 +936,7 @@ async def test_bulk_stream_error_in_logs_on_create_job(stream_config, stream_api ], ) async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, status_code, response_json, error_message): - stream = await generate_stream("Account", stream_config, stream_api) + stream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() url = f"{stream.sf_api.instance_url}/services/data/{stream.sf_api.version}/jobs/query/queryJobId" @@ -951,7 +951,7 @@ async def test_bulk_stream_error_on_wait_for_job(stream_config, stream_api, stat @pytest.mark.asyncio() @freezegun.freeze_time("2023-01-01") async def test_bulk_stream_slices(stream_config_date_format, stream_api): - stream: BulkIncrementalSalesforceStream = await generate_stream("FakeBulkStream", stream_config_date_format, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("FakeBulkStream", stream_config_date_format, stream_api) stream_slices = [s async for s in stream.stream_slices(sync_mode=SyncMode.full_refresh)] expected_slices = [] today = pendulum.today(tz="UTC") @@ -971,10 +971,10 @@ async def test_bulk_stream_slices(stream_config_date_format, stream_api): @freezegun.freeze_time("2023-04-01") async def test_bulk_stream_request_params_states(stream_config_date_format, stream_api, bulk_catalog): stream_config_date_format.update({"start_date": "2023-01-01"}) - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config_date_format, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config_date_format, stream_api) await stream.ensure_session() - source = SalesforceSourceDispatcher(SourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) + source = SalesforceSourceDispatcher(AsyncSourceSalesforce(_ANY_CATALOG, _ANY_CONFIG)) source.streams = Mock() source.streams.return_value = [stream] base_url = f"{stream.sf_api.instance_url}{stream.path()}" @@ -1034,7 +1034,7 @@ async def test_bulk_stream_request_params_states(stream_config_date_format, stre @pytest.mark.asyncio async def test_request_params_incremental(stream_config_date_format, stream_api): - stream = await generate_stream("ContentDocument", stream_config_date_format, stream_api) + stream = await generate_stream_async("ContentDocument", stream_config_date_format, stream_api) params = stream.request_params(stream_state={}, stream_slice={'start_date': '2020', 'end_date': '2021'}) assert params == {'q': 'SELECT LastModifiedDate, Id FROM ContentDocument WHERE LastModifiedDate >= 2020 AND LastModifiedDate < 2021'} @@ -1042,7 +1042,7 @@ async def test_request_params_incremental(stream_config_date_format, stream_api) @pytest.mark.asyncio async def test_request_params_substream(stream_config_date_format, stream_api): - stream = await generate_stream("ContentDocumentLink", stream_config_date_format, stream_api) + stream = await generate_stream_async("ContentDocumentLink", stream_config_date_format, stream_api) params = stream.request_params(stream_state={}, stream_slice={'parents': [{'Id': 1}, {'Id': 2}]}) assert params == {"q": "SELECT LastModifiedDate, Id FROM ContentDocumentLink WHERE ContentDocumentId IN ('1','2')"} @@ -1052,7 +1052,7 @@ async def test_request_params_substream(stream_config_date_format, stream_api): @freezegun.freeze_time("2023-03-20") async def test_stream_slices_for_substream(stream_config, stream_api): stream_config['start_date'] = '2023-01-01' - stream: BulkSalesforceSubStream = await generate_stream("ContentDocumentLink", stream_config, stream_api) + stream: BulkSalesforceSubStream = await generate_stream_async("ContentDocumentLink", stream_config, stream_api) stream.SLICE_BATCH_SIZE = 2 # each ContentDocumentLink should contain 2 records from parent ContentDocument stream await stream.ensure_session() diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py index cb0689e24061..da7ccbed50fc 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/conftest.py @@ -7,8 +7,9 @@ import pytest from airbyte_cdk.models import ConfiguredAirbyteCatalog -from source_salesforce.async_salesforce.api import Salesforce +from source_salesforce.api import Salesforce from source_salesforce.async_salesforce.source import AsyncSourceSalesforce +from source_salesforce.source import SourceSalesforce @pytest.fixture(autouse=True) @@ -119,7 +120,11 @@ def stream_api_v2_pk_too_many_properties(stream_config): return _stream_api(stream_config, describe_response_data=describe_response_data) -async def generate_stream(stream_name, stream_config, stream_api): +def generate_stream(stream_name, stream_config, stream_api): + return (SourceSalesforce.generate_streams(stream_config, {stream_name: None}, stream_api))[0] + + +async def generate_stream_async(stream_name, stream_config, stream_api): return (await AsyncSourceSalesforce.generate_streams(stream_config, {stream_name: None}, stream_api))[0] diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py index 42dfdcb047be..f0151e97b036 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/discovery_test.py @@ -5,8 +5,8 @@ from unittest.mock import Mock import pytest -from source_salesforce.async_salesforce.api import DATE_TYPES, LOOSE_TYPES, NUMBER_TYPES, STRING_TYPES, Salesforce -from source_salesforce.async_salesforce.exceptions import TypeSalesforceException +from source_salesforce.api import DATE_TYPES, LOOSE_TYPES, NUMBER_TYPES, STRING_TYPES, Salesforce +from source_salesforce.exceptions import TypeSalesforceException @pytest.mark.parametrize( diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py index 510c19494e75..75780d693822 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory.py @@ -6,9 +6,9 @@ import tracemalloc import pytest -from aioresponses import CallbackResult, aioresponses +import requests_mock from conftest import generate_stream -from source_salesforce.async_salesforce.streams import BulkIncrementalSalesforceStream +from source_salesforce.streams import BulkIncrementalSalesforceStream @pytest.mark.parametrize( @@ -26,22 +26,17 @@ "200k records", ], ) -@pytest.mark.asyncio -async def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): +def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) - await stream.ensure_session() + stream: BulkIncrementalSalesforceStream = generate_stream("Account", stream_config, stream_api) content = b'"Id","IsDeleted"' for _ in range(n_records): content += b'"0014W000027f6UwQAI","false"\n' - def callback(url, **kwargs): - return CallbackResult(body=content) - - with aioresponses() as m: - m.get(job_full_url_results, status=200, callback=callback) + with requests_mock.Mocker() as m: + m.register_uri("GET", job_full_url_results, content=content) tracemalloc.start() - tmp_file, response_encoding, _ = await stream.download_data(url=job_full_url_results) + tmp_file, response_encoding, _ = stream.download_data(url=job_full_url_results) for x in stream.read_with_chunks(tmp_file, response_encoding): pass fs, fp = tracemalloc.get_traced_memory() @@ -49,5 +44,3 @@ def callback(url, **kwargs): assert first_size_in_mb < first_size assert first_peak_in_mb < first_peak - - await stream._session.close() diff --git a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py index 510c19494e75..1c79fe5bc75a 100644 --- a/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py +++ b/airbyte-integrations/connectors/source-salesforce/unit_tests/test_memory_async.py @@ -7,7 +7,7 @@ import pytest from aioresponses import CallbackResult, aioresponses -from conftest import generate_stream +from conftest import generate_stream_async from source_salesforce.async_salesforce.streams import BulkIncrementalSalesforceStream @@ -29,7 +29,7 @@ @pytest.mark.asyncio async def test_memory_download_data(stream_config, stream_api, n_records, first_size, first_peak): job_full_url_results: str = "https://fase-account.salesforce.com/services/data/v57.0/jobs/query/7504W00000bkgnpQAA/results" - stream: BulkIncrementalSalesforceStream = await generate_stream("Account", stream_config, stream_api) + stream: BulkIncrementalSalesforceStream = await generate_stream_async("Account", stream_config, stream_api) await stream.ensure_session() content = b'"Id","IsDeleted"' for _ in range(n_records): From d6cfc45ecebdde73761a2bf4faccb4646461427d Mon Sep 17 00:00:00 2001 From: Catherine Noll Date: Mon, 8 Jan 2024 22:40:43 -0500 Subject: [PATCH 74/74] Salesforce async full refresh & sync incremental --- .../sources/async_cdk/source_dispatcher.py | 133 ++++++++++++++++-- .../connectors/source-salesforce/main.py | 11 +- .../source_salesforce/__init__.py | 3 +- 3 files changed, 127 insertions(+), 20 deletions(-) diff --git a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py index c79b4cb3c813..18dd323a7c94 100644 --- a/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py +++ b/airbyte-cdk/python/airbyte_cdk/sources/async_cdk/source_dispatcher.py @@ -25,12 +25,14 @@ ConfiguredAirbyteCatalog, ConfiguredAirbyteStream, ConnectorSpecification, + SyncMode, ) from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager from airbyte_cdk.sources.abstract_source import AbstractSource from airbyte_cdk.sources.async_cdk.abstract_source_async import AsyncAbstractSource from airbyte_cdk.sources.async_cdk.source_reader import Sentinel, SourceReader from airbyte_cdk.sources.async_cdk.streams.core_async import AsyncStream +from airbyte_cdk.sources.streams import Stream from airbyte_cdk.sources.utils.schema_helpers import InternalConfig, split_config from airbyte_cdk.utils.event_timing import create_timer from airbyte_cdk.utils.traced_exception import AirbyteTracedException @@ -48,8 +50,9 @@ class SourceDispatcher(AbstractSource, ABC): Abstract base class for an Airbyte Source that can dispatch to an async source. """ - def __init__(self, async_source: AsyncAbstractSource): + def __init__(self, async_source: AsyncAbstractSource, source: AbstractSource): self.async_source = async_source + self.source = source self.queue = Queue(DEFAULT_QUEUE_SIZE) self.session_limit = DEFAULT_SESSION_LIMIT @@ -82,14 +85,8 @@ def read( logger: logging.Logger, config: Mapping[str, Any], catalog: ConfiguredAirbyteCatalog, - state: Optional[ - Union[List[AirbyteStateMessage], MutableMapping[str, Any]] - ] = None, - ) -> Iterator[AirbyteMessage]: - """ - Run the async_source's `read_streams` method and yield its results. - - """ + state: Optional[Union[List[AirbyteStateMessage], MutableMapping[str, Any]]] = None, + ): logger.info(f"Starting syncing {self.name}") config, internal_config = split_config(config) stream_instances: Mapping[str, AsyncStream] = { @@ -100,19 +97,42 @@ def read( ) self._stream_to_instance_map = stream_instances self._assert_streams(catalog, stream_instances) - n_records = 0 + stream_name_to_exception: MutableMapping[str, AirbyteTracedException] = {} # TODO: wire this option through for asyncio + full_refresh_streams = {} + incremental_streams = {} + + for stream in catalog.streams: + if stream.sync_mode == SyncMode.full_refresh: + full_refresh_streams[stream.stream.name] = stream_instances[stream.stream.name] + else: + incremental_streams[stream.stream.name] = stream_instances[stream.stream.name] + with create_timer(self.name) as timer: - for record in self._do_read( - catalog, stream_instances, timer, logger, state_manager, internal_config + for record in self._read_async_source( + catalog, full_refresh_streams, timer, logger, state_manager, internal_config + ): + n_records += 1 + yield record + + for record in self._read_sync_source( + catalog, incremental_streams, timer, logger, state_manager, internal_config, stream_name_to_exception ): n_records += 1 yield record print(f"_______________________-ASYNCIO SOURCE N RECORDS == {n_records}") + + if self.continue_sync_on_stream_failure and len(stream_name_to_exception) > 0: + raise AirbyteTracedException( + message=self._generate_failed_streams_error_message( + stream_name_to_exception + ) + ) + logger.info(f"Finished syncing {self.name}") - def _do_read( + def _read_sync_source( self, catalog: ConfiguredAirbyteCatalog, stream_instances: Dict[str, AsyncStream], @@ -120,7 +140,94 @@ def _do_read( logger: logging.Logger, state_manager: ConnectorStateManager, internal_config: InternalConfig, + stream_name_to_exception: MutableMapping[str, AirbyteTracedException], ): + """ + For concurrent streams, records from the sync source. + + TODO: this can be deleted when asyncio is deployed for incremental + """ + for configured_stream in catalog.streams: + stream_instance = stream_instances.get(configured_stream.stream.name) + if not stream_instance: + if not self.raise_exception_on_missing_stream: + continue + raise KeyError( + f"The stream {configured_stream.stream.name} no longer exists in the configuration. " + f"Refresh the schema in replication settings and remove this stream from future sync attempts." + ) + + try: + timer.start_event(f"Syncing stream {configured_stream.stream.name}") + stream_is_available, reason = stream_instance.check_availability( + logger, self + ) + if not stream_is_available: + logger.warning( + f"Skipped syncing stream '{stream_instance.name}' because it was unavailable. {reason}" + ) + continue + logger.info( + f"Marking stream {configured_stream.stream.name} as STARTED" + ) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.STARTED + ) + yield from self._read_stream( + logger=logger, + stream_instance=stream_instance, + configured_stream=configured_stream, + state_manager=state_manager, + internal_config=internal_config, + ) + logger.info( + f"Marking stream {configured_stream.stream.name} as STOPPED" + ) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.COMPLETE + ) + except AirbyteTracedException as e: + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.INCOMPLETE + ) + if self.continue_sync_on_stream_failure: + stream_name_to_exception[stream_instance.name] = e + else: + raise e + except Exception as e: + yield from self._emit_queued_messages() + logger.exception( + f"Encountered an exception while reading stream {configured_stream.stream.name}" + ) + logger.info( + f"Marking stream {configured_stream.stream.name} as STOPPED" + ) + yield stream_status_as_airbyte_message( + configured_stream.stream, AirbyteStreamStatus.INCOMPLETE + ) + display_message = stream_instance.get_error_display_message(e) + if display_message: + raise AirbyteTracedException.from_exception( + e, message=display_message + ) from e + raise e + finally: + timer.finish_event() + logger.info(f"Finished syncing {configured_stream.stream.name}") + logger.info(timer.report()) + + def _read_async_source( + self, + catalog: ConfiguredAirbyteCatalog, + stream_instances: Dict[str, AsyncStream], + timer: Any, + logger: logging.Logger, + state_manager: ConnectorStateManager, + internal_config: InternalConfig, + ) -> Iterator[AirbyteMessage]: + """ + Run the async_source's `read_streams` method and yield its results. + """ streams_in_progress_sentinels = { s.stream.name: Sentinel(s.stream.name) for s in catalog.streams diff --git a/airbyte-integrations/connectors/source-salesforce/main.py b/airbyte-integrations/connectors/source-salesforce/main.py index f2f93f940e73..aa92e9371817 100644 --- a/airbyte-integrations/connectors/source-salesforce/main.py +++ b/airbyte-integrations/connectors/source-salesforce/main.py @@ -10,18 +10,18 @@ from airbyte_cdk.entrypoint import AirbyteEntrypoint, launch from airbyte_cdk.models import AirbyteErrorTraceMessage, AirbyteMessage, AirbyteTraceMessage, TraceType, Type -from source_salesforce import SalesforceSourceDispatcher, AsyncSourceSalesforce +from source_salesforce import AsyncSourceSalesforce, SalesforceSourceDispatcher, SourceSalesforce def _get_source(args: List[str]): catalog_path = AirbyteEntrypoint.extract_catalog(args) config_path = AirbyteEntrypoint.extract_config(args) + catalog = AsyncSourceSalesforce.read_catalog(catalog_path) if catalog_path else None + config = AsyncSourceSalesforce.read_config(config_path) if config_path else None try: return SalesforceSourceDispatcher( - AsyncSourceSalesforce( - AsyncSourceSalesforce.read_catalog(catalog_path) if catalog_path else None, - AsyncSourceSalesforce.read_config(config_path) if config_path else None, - ) + AsyncSourceSalesforce(catalog, config), + SourceSalesforce(catalog, config) ) except Exception as error: print( @@ -41,7 +41,6 @@ def _get_source(args: List[str]): if __name__ == "__main__": - # _args = ['read', '--config', 'secrets/config.json', '--catalog', 'integration_tests/configured_catalog.json'] _args = sys.argv[1:] source = _get_source(_args) if source: diff --git a/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py b/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py index b410c64d63c2..9cb637717902 100644 --- a/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py +++ b/airbyte-integrations/connectors/source-salesforce/source_salesforce/__init__.py @@ -3,5 +3,6 @@ # from .async_salesforce.source import AsyncSourceSalesforce, SalesforceSourceDispatcher +from .source import SourceSalesforce -__all__ = ["AsyncSourceSalesforce", "SalesforceSourceDispatcher"] +__all__ = ["AsyncSourceSalesforce", "SalesforceSourceDispatcher", "AsyncSourceSalesforce"]