Skip to content

Commit

Permalink
[RFR for API Sources] New Python interfaces to support resumable full…
Browse files Browse the repository at this point in the history
… refresh (#37429)
  • Loading branch information
brianjlai committed May 6, 2024
1 parent 3cca1c0 commit d74125b
Show file tree
Hide file tree
Showing 24 changed files with 1,655 additions and 232 deletions.
19 changes: 3 additions & 16 deletions airbyte-cdk/python/airbyte_cdk/sources/abstract_source.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,12 @@
FailureType,
Status,
StreamDescriptor,
SyncMode,
)
from airbyte_cdk.models import Type as MessageType
from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager
from airbyte_cdk.sources.message import InMemoryMessageRepository, MessageRepository
from airbyte_cdk.sources.source import Source
from airbyte_cdk.sources.streams import FULL_REFRESH_SENTINEL_STATE_KEY, Stream
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
Expand Down Expand Up @@ -211,15 +210,9 @@ def _read_stream(
stream_instance.log_stream_sync_configuration()

stream_name = configured_stream.stream.name
# The platform always passes stream state regardless of sync mode. We shouldn't need to consider this case within the
# connector, but right now we need to prevent accidental usage of the previous stream state
stream_state = (
state_manager.get_stream_state(stream_name, stream_instance.namespace)
if configured_stream.sync_mode == SyncMode.incremental
else {}
)
stream_state = state_manager.get_stream_state(stream_name, stream_instance.namespace)

if stream_state and "state" in dir(stream_instance) and not self._stream_state_is_full_refresh(stream_state):
if "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}")

Expand Down Expand Up @@ -275,9 +268,3 @@ def stop_sync_on_stream_failure(self) -> bool:
on the first error seen and emit a single error trace message for that stream.
"""
return False

@staticmethod
def _stream_state_is_full_refresh(stream_state: Mapping[str, Any]) -> bool:
# For full refresh syncs that don't have a suitable cursor value, we emit a state that contains a sentinel key.
# This key is never used by a connector and is needed during a read to skip assigning the incoming state.
return FULL_REFRESH_SENTINEL_STATE_KEY in stream_state
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,12 @@ def cursor_field(self) -> Union[str, List[str]]:
cursor = self._stream_cursor_field.eval(self.config)
return cursor if cursor else []

@property
def is_resumable(self) -> bool:
# Declarative sources always implement state getter/setter, but whether it supports checkpointing is based on
# if the retriever has a cursor defined.
return self.retriever.cursor is not None if hasattr(self.retriever, "cursor") else False

def read_records(
self,
sync_mode: SyncMode,
Expand All @@ -108,7 +114,7 @@ def read_records(
"""
:param: stream_state We knowingly avoid using stream_state as we want cursors to manage their own state.
"""
if stream_slice is None:
if stream_slice is None or stream_slice == {}:
# As the parameter is Optional, many would just call `read_records(sync_mode)` during testing without specifying the field
# As part of the declarative model without custom components, this should never happen as the CDK would wire up a
# SinglePartitionRouter that would create this StreamSlice properly
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
from airbyte_cdk.sources.file_based.stream.concurrent.cursor.abstract_concurrent_file_based_cursor import AbstractConcurrentFileBasedCursor
from airbyte_cdk.sources.file_based.types import StreamState
from airbyte_cdk.sources.message import MessageRepository
from airbyte_cdk.sources.streams import FULL_REFRESH_SENTINEL_STATE_KEY
from airbyte_cdk.sources.streams import NO_CURSOR_STATE_KEY
from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition
from airbyte_cdk.sources.streams.concurrent.partitions.record import Record

Expand All @@ -36,7 +36,7 @@ def __init__(

@property
def state(self) -> MutableMapping[str, Any]:
return {FULL_REFRESH_SENTINEL_STATE_KEY: True}
return {NO_CURSOR_STATE_KEY: True}

def observe(self, record: Record) -> None:
pass
Expand Down
4 changes: 2 additions & 2 deletions airbyte-cdk/python/airbyte_cdk/sources/streams/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,6 @@
#

# Initialize Streams Package
from .core import FULL_REFRESH_SENTINEL_STATE_KEY, IncrementalMixin, Stream
from .core import NO_CURSOR_STATE_KEY, IncrementalMixin, CheckpointMixin, Stream

__all__ = ["FULL_REFRESH_SENTINEL_STATE_KEY", "IncrementalMixin", "Stream"]
__all__ = ["NO_CURSOR_STATE_KEY", "IncrementalMixin", "CheckpointMixin", "Stream"]
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
# Copyright (c) 2024 Airbyte, Inc., all rights reserved.


from .checkpoint_reader import CheckpointMode, CheckpointReader, FullRefreshCheckpointReader, IncrementalCheckpointReader, ResumableFullRefreshCheckpointReader

__all__ = ["CheckpointMode", "CheckpointReader", "FullRefreshCheckpointReader", "IncrementalCheckpointReader", "ResumableFullRefreshCheckpointReader"]
Original file line number Diff line number Diff line change
@@ -0,0 +1,126 @@
# Copyright (c) 2024 Airbyte, Inc., all rights reserved.

from abc import ABC, abstractmethod
from enum import Enum
from typing import Any, Iterable, Mapping, Optional


class CheckpointMode(Enum):
INCREMENTAL = "incremental"
RESUMABLE_FULL_REFRESH = "resumable_full_refresh"
FULL_REFRESH = "full_refresh"


class CheckpointReader(ABC):
"""
CheckpointReader manages how to iterate over a stream's partitions and serves as the bridge for interpreting the current state
of the stream that should be emitted back to the platform.
"""

@abstractmethod
def next(self) -> Optional[Mapping[str, Any]]:
"""
Returns the next slice that will be used to fetch the next group of records. Returning None indicates that the reader
has finished iterating over all slices.
"""

@abstractmethod
def observe(self, new_state: Mapping[str, Any]) -> None:
"""
Updates the internal state of the checkpoint reader based on the incoming stream state from a connector.
WARNING: This is used to retain backwards compatibility with streams using the legacy get_stream_state() method.
In order to uptake Resumable Full Refresh, connectors must migrate streams to use the state setter/getter methods.
"""

@abstractmethod
def get_checkpoint(self) -> Optional[Mapping[str, Any]]:
"""
Retrieves the current state value of the stream. The connector does not emit state messages if the checkpoint value is None.
"""


class IncrementalCheckpointReader(CheckpointReader):
"""
IncrementalCheckpointReader handles iterating through a stream based on partitioned windows of data that are determined
before syncing data.
"""

def __init__(self, stream_state: Mapping[str, Any], stream_slices: Iterable[Optional[Mapping[str, Any]]]):
self._state: Optional[Mapping[str, Any]] = stream_state
self._stream_slices = iter(stream_slices)
self._has_slices = False

def next(self) -> Optional[Mapping[str, Any]]:
try:
next_slice = next(self._stream_slices)
self._has_slices = True
return next_slice
except StopIteration:
# This is used to avoid sending a duplicate state message at the end of a sync since the stream has already
# emitted state at the end of each slice. If we want to avoid this extra complexity, we can also just accept
# that every sync emits a final duplicate state
if self._has_slices:
self._state = None
return None

def observe(self, new_state: Mapping[str, Any]) -> None:
self._state = new_state

def get_checkpoint(self) -> Optional[Mapping[str, Any]]:
return self._state


class ResumableFullRefreshCheckpointReader(CheckpointReader):
"""
ResumableFullRefreshCheckpointReader allows for iteration over an unbounded set of records based on the pagination strategy
of the stream. Because the number of pages is unknown, the stream's current state is used to determine whether to continue
fetching more pages or stopping the sync.
"""

def __init__(self, stream_state: Mapping[str, Any]):
# The first attempt of an RFR stream has an empty {} incoming state, but should still make a first attempt to read records
# from the first page in next().
self._first_page = bool(stream_state == {})
self._state: Mapping[str, Any] = stream_state

def next(self) -> Optional[Mapping[str, Any]]:
if self._first_page:
self._first_page = False
return self._state
elif self._state == {}:
return None
else:
return self._state

def observe(self, new_state: Mapping[str, Any]) -> None:
self._state = new_state

def get_checkpoint(self) -> Optional[Mapping[str, Any]]:
return self._state or {}


class FullRefreshCheckpointReader(CheckpointReader):
"""
FullRefreshCheckpointReader iterates over data that cannot be checkpointed incrementally during the sync because the stream
is not capable of managing state. At the end of a sync, a final state message is emitted to signal completion.
"""

def __init__(self, stream_slices: Iterable[Optional[Mapping[str, Any]]]):
self._stream_slices = iter(stream_slices)
self._final_checkpoint = False

def next(self) -> Optional[Mapping[str, Any]]:
try:
return next(self._stream_slices)
except StopIteration:
self._final_checkpoint = True
return None

def observe(self, new_state: Mapping[str, Any]) -> None:
pass

def get_checkpoint(self) -> Optional[Mapping[str, Any]]:
if self._final_checkpoint:
return {"__ab_no_cursor_state_message": True}
return None
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@

from airbyte_cdk.sources.connector_state_manager import ConnectorStateManager
from airbyte_cdk.sources.message import MessageRepository
from airbyte_cdk.sources.streams import FULL_REFRESH_SENTINEL_STATE_KEY
from airbyte_cdk.sources.streams import NO_CURSOR_STATE_KEY
from airbyte_cdk.sources.streams.concurrent.partitions.partition import Partition
from airbyte_cdk.sources.streams.concurrent.partitions.record import Record
from airbyte_cdk.sources.streams.concurrent.state_converters.abstract_stream_state_converter import AbstractStreamStateConverter
Expand Down Expand Up @@ -107,7 +107,7 @@ def __init__(

@property
def state(self) -> MutableMapping[str, Any]:
return {FULL_REFRESH_SENTINEL_STATE_KEY: True}
return {NO_CURSOR_STATE_KEY: True}

def observe(self, record: Record) -> None:
pass
Expand Down
Loading

0 comments on commit d74125b

Please sign in to comment.