diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 3dff9219..eb7a2021 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -40,6 +40,8 @@ jobs: - name: Run tests run: make test-all + env: + LD_SKIP_FLAKY_TESTS: true - name: Verify typehints run: make lint @@ -92,3 +94,5 @@ jobs: - name: Run tests run: make test-all + env: + LD_SKIP_FLAKY_TESTS: true diff --git a/Makefile b/Makefile index 9ee4463d..f2cc2cbb 100644 --- a/Makefile +++ b/Makefile @@ -38,6 +38,7 @@ test-all: install .PHONY: lint lint: #! Run type analysis and linting checks lint: install + @mkdir -p .mypy_cache @poetry run mypy ldclient @poetry run isort --check --atomic ldclient contract-tests @poetry run pycodestyle ldclient contract-tests diff --git a/contract-tests/client_entity.py b/contract-tests/client_entity.py index c0030adb..f68f7488 100644 --- a/contract-tests/client_entity.py +++ b/contract-tests/client_entity.py @@ -15,6 +15,13 @@ Stage ) from ldclient.config import BigSegmentsConfig +from ldclient.datasystem import ( + custom, + fdv1_fallback_ds_builder, + polling_ds_builder, + streaming_ds_builder +) +from ldclient.impl.datasourcev2.polling import PollingDataSourceBuilder class ClientEntity: @@ -29,7 +36,75 @@ def __init__(self, tag, config): 'version': tags.get('applicationVersion', ''), } - if config.get("streaming") is not None: + datasystem_config = config.get('dataSystem') + if datasystem_config is not None: + datasystem = custom() + + init_configs = datasystem_config.get('initializers') + if init_configs is not None: + initializers = [] + for init_config in init_configs: + polling = init_config.get('polling') + if polling is not None: + if polling.get("baseUri") is not None: + opts["base_uri"] = polling["baseUri"] + _set_optional_time_prop(polling, "pollIntervalMs", opts, "poll_interval") + polling = polling_ds_builder() + initializers.append(polling) + + datasystem.initializers(initializers) + sync_config = datasystem_config.get('synchronizers') + if sync_config is not None: + primary = sync_config.get('primary') + secondary = sync_config.get('secondary') + + primary_builder = None + secondary_builder = None + fallback_builder = None + + if primary is not None: + streaming = primary.get('streaming') + if streaming is not None: + primary_builder = streaming_ds_builder() + if streaming.get("baseUri") is not None: + opts["stream_uri"] = streaming["baseUri"] + _set_optional_time_prop(streaming, "initialRetryDelayMs", opts, "initial_reconnect_delay") + primary_builder = streaming_ds_builder() + elif primary.get('polling') is not None: + polling = primary.get('polling') + if polling.get("baseUri") is not None: + opts["base_uri"] = polling["baseUri"] + _set_optional_time_prop(polling, "pollIntervalMs", opts, "poll_interval") + primary_builder = polling_ds_builder() + fallback_builder = fdv1_fallback_ds_builder() + + if secondary is not None: + streaming = secondary.get('streaming') + if streaming is not None: + secondary_builder = streaming_ds_builder() + if streaming.get("baseUri") is not None: + opts["stream_uri"] = streaming["baseUri"] + _set_optional_time_prop(streaming, "initialRetryDelayMs", opts, "initial_reconnect_delay") + secondary_builder = streaming_ds_builder() + elif secondary.get('polling') is not None: + polling = secondary.get('polling') + if polling.get("baseUri") is not None: + opts["base_uri"] = polling["baseUri"] + _set_optional_time_prop(polling, "pollIntervalMs", opts, "poll_interval") + secondary_builder = polling_ds_builder() + fallback_builder = fdv1_fallback_ds_builder() + + if primary_builder is not None: + datasystem.synchronizers(primary_builder, secondary_builder) + if fallback_builder is not None: + datasystem.fdv1_compatible_synchronizer(fallback_builder) + + if datasystem_config.get("payloadFilter") is not None: + opts["payload_filter_key"] = datasystem_config["payloadFilter"] + + opts["datasystem_config"] = datasystem.build() + + elif config.get("streaming") is not None: streaming = config["streaming"] if streaming.get("baseUri") is not None: opts["stream_uri"] = streaming["baseUri"] diff --git a/ldclient/client.py b/ldclient/client.py index 091b064f..7022f137 100644 --- a/ldclient/client.py +++ b/ldclient/client.py @@ -30,6 +30,8 @@ DataStoreStatusProviderImpl, DataStoreUpdateSinkImpl ) +from ldclient.impl.datasystem import DataAvailability, DataSystem +from ldclient.impl.datasystem.fdv2 import FDv2 from ldclient.impl.evaluator import Evaluator, error_reason from ldclient.impl.events.diagnostics import ( _DiagnosticAccumulator, @@ -51,7 +53,8 @@ DataStoreStatusProvider, DataStoreUpdateSink, FeatureStore, - FlagTracker + FlagTracker, + ReadOnlyStore ) from ldclient.migrations import OpTracker, Stage from ldclient.plugin import ( @@ -249,14 +252,19 @@ def __start_up(self, start_wait: float): self.__hooks_lock = ReadWriteLock() self.__hooks = self._config.hooks + plugin_hooks # type: List[Hook] - # Initialize data system (FDv1) to encapsulate v1 data plumbing - from ldclient.impl.datasystem.fdv1 import ( # local import to avoid circular dependency - FDv1 - ) + datasystem_config = self._config.datasystem_config + if datasystem_config is None: + # Initialize data system (FDv1) to encapsulate v1 data plumbing + from ldclient.impl.datasystem.fdv1 import ( # local import to avoid circular dependency + FDv1 + ) + + self._data_system: DataSystem = FDv1(self._config) + else: + self._data_system = FDv2(self._config, datasystem_config) - self._data_system = FDv1(self._config) # Provide flag evaluation function for value-change tracking - self._data_system.set_flag_value_eval_fn( + self._data_system.set_flag_value_eval_fn( # type: ignore lambda key, context: self.variation(key, context, None) ) # Expose providers and store from data system @@ -265,14 +273,13 @@ def __start_up(self, start_wait: float): self._data_system.data_source_status_provider ) self.__flag_tracker = self._data_system.flag_tracker - self._store = self._data_system.store # type: FeatureStore big_segment_store_manager = BigSegmentStoreManager(self._config.big_segments) self.__big_segment_store_manager = big_segment_store_manager self._evaluator = Evaluator( - lambda key: _get_store_item(self._store, FEATURES, key), - lambda key: _get_store_item(self._store, SEGMENTS, key), + lambda key: _get_store_item(self._data_system.store, FEATURES, key), + lambda key: _get_store_item(self._data_system.store, SEGMENTS, key), lambda key: big_segment_store_manager.get_user_membership(key), log, ) @@ -286,7 +293,7 @@ def __start_up(self, start_wait: float): diagnostic_accumulator = self._set_event_processor(self._config) # Pass diagnostic accumulator to data system for streaming metrics - self._data_system.set_diagnostic_accumulator(diagnostic_accumulator) + self._data_system.set_diagnostic_accumulator(diagnostic_accumulator) # type: ignore self.__register_plugins(environment_metadata) @@ -475,11 +482,7 @@ def is_initialized(self) -> bool: if self.is_offline() or self._config.use_ldd: return True - return ( - self._data_system._update_processor.initialized() - if self._data_system._update_processor - else False - ) + return self._data_system.data_availability.at_least(DataAvailability.CACHED) def flush(self): """Flushes all pending analytics events. @@ -567,7 +570,7 @@ def _evaluate_internal(self, key: str, context: Context, default: Any, event_fac return EvaluationDetail(default, None, error_reason('CLIENT_NOT_READY')), None if not self.is_initialized(): - if self._store.initialized: + if self._data_system.store.initialized: log.warning("Feature Flag evaluation attempted before client has initialized - using last known values from feature store for feature key: " + key) else: log.warning("Feature Flag evaluation attempted before client has initialized! Feature store unavailable - returning default: " + str(default) + " for feature key: " + key) @@ -580,7 +583,7 @@ def _evaluate_internal(self, key: str, context: Context, default: Any, event_fac return EvaluationDetail(default, None, error_reason('USER_NOT_SPECIFIED')), None try: - flag = _get_store_item(self._store, FEATURES, key) + flag = _get_store_item(self._data_system.store, FEATURES, key) except Exception as e: log.error("Unexpected error while retrieving feature flag \"%s\": %s" % (key, repr(e))) log.debug(traceback.format_exc()) @@ -638,7 +641,7 @@ def all_flags_state(self, context: Context, **kwargs) -> FeatureFlagsState: return FeatureFlagsState(False) if not self.is_initialized(): - if self._store.initialized: + if self._data_system.store.initialized: log.warning("all_flags_state() called before client has finished initializing! Using last known values from feature store") else: log.warning("all_flags_state() called before client has finished initializing! Feature store unavailable - returning empty state") @@ -653,7 +656,7 @@ def all_flags_state(self, context: Context, **kwargs) -> FeatureFlagsState: with_reasons = kwargs.get('with_reasons', False) details_only_if_tracked = kwargs.get('details_only_for_tracked_flags', False) try: - flags_map = self._store.all(FEATURES, lambda x: x) + flags_map = self._data_system.store.all(FEATURES, lambda x: x) if flags_map is None: raise ValueError("feature store error") except Exception as e: diff --git a/ldclient/config.py b/ldclient/config.py index fbc88ac8..8e5caf76 100644 --- a/ldclient/config.py +++ b/ldclient/config.py @@ -4,8 +4,9 @@ Note that the same class can also be imported from the ``ldclient.client`` submodule. """ +from dataclasses import dataclass from threading import Event -from typing import Callable, List, Optional, Set +from typing import Callable, List, Optional, Set, TypeVar from ldclient.feature_store import InMemoryFeatureStore from ldclient.hook import Hook @@ -17,8 +18,11 @@ from ldclient.interfaces import ( BigSegmentStore, DataSourceUpdateSink, + DataStoreMode, EventProcessor, FeatureStore, + Initializer, + Synchronizer, UpdateProcessor ) from ldclient.plugin import Plugin @@ -152,6 +156,34 @@ def disable_ssl_verification(self) -> bool: return self.__disable_ssl_verification +T = TypeVar("T") + +Builder = Callable[['Config'], T] + + +@dataclass(frozen=True) +class DataSystemConfig: + """Configuration for LaunchDarkly's data acquisition strategy.""" + + initializers: Optional[List[Builder[Initializer]]] + """The initializers for the data system.""" + + primary_synchronizer: Optional[Builder[Synchronizer]] + """The primary synchronizer for the data system.""" + + secondary_synchronizer: Optional[Builder[Synchronizer]] = None + """The secondary synchronizers for the data system.""" + + data_store_mode: DataStoreMode = DataStoreMode.READ_WRITE + """The data store mode specifies the mode in which the persistent store will operate, if present.""" + + data_store: Optional[FeatureStore] = None + """The (optional) persistent data store instance.""" + + fdv1_fallback_synchronizer: Optional[Builder[Synchronizer]] = None + """An optional fallback synchronizer that will read from FDv1""" + + class Config: """Advanced configuration options for the SDK client. @@ -194,6 +226,7 @@ def __init__( enable_event_compression: bool = False, omit_anonymous_contexts: bool = False, payload_filter_key: Optional[str] = None, + datasystem_config: Optional[DataSystemConfig] = None, ): """ :param sdk_key: The SDK key for your LaunchDarkly account. This is always required. @@ -264,6 +297,7 @@ def __init__( :param enable_event_compression: Whether or not to enable GZIP compression for outgoing events. :param omit_anonymous_contexts: Sets whether anonymous contexts should be omitted from index and identify events. :param payload_filter_key: The payload filter is used to selectively limited the flags and segments delivered in the data source payload. + :param datasystem_config: Configuration for the upcoming enhanced data system design. This is experimental and should not be set without direction from LaunchDarkly support. """ self.__sdk_key = validate_sdk_key_format(sdk_key, log) @@ -303,6 +337,7 @@ def __init__( self.__payload_filter_key = payload_filter_key self._data_source_update_sink: Optional[DataSourceUpdateSink] = None self._instance_id: Optional[str] = None + self._datasystem_config = datasystem_config def copy_with_new_sdk_key(self, new_sdk_key: str) -> 'Config': """Returns a new ``Config`` instance that is the same as this one, except for having a different SDK key. @@ -546,6 +581,15 @@ def data_source_update_sink(self) -> Optional[DataSourceUpdateSink]: """ return self._data_source_update_sink + @property + def datasystem_config(self) -> Optional[DataSystemConfig]: + """ + Configuration for the upcoming enhanced data system design. This is + experimental and should not be set without direction from LaunchDarkly + support. + """ + return self._datasystem_config + def _validate(self): if self.offline is False and self.sdk_key == '': log.warning("Missing or blank SDK key") diff --git a/ldclient/datasystem.py b/ldclient/datasystem.py new file mode 100644 index 00000000..89a15e11 --- /dev/null +++ b/ldclient/datasystem.py @@ -0,0 +1,216 @@ +""" +Configuration for LaunchDarkly's data acquisition strategy. +""" + +from typing import Callable, List, Optional, TypeVar + +from ldclient.config import Config as LDConfig +from ldclient.config import DataSystemConfig +from ldclient.impl.datasourcev2.polling import ( + PollingDataSource, + PollingDataSourceBuilder, + Urllib3FDv1PollingRequester, + Urllib3PollingRequester +) +from ldclient.impl.datasourcev2.streaming import ( + StreamingDataSource, + StreamingDataSourceBuilder +) +from ldclient.interfaces import ( + DataStoreMode, + FeatureStore, + Initializer, + Synchronizer +) + +T = TypeVar("T") + +Builder = Callable[[LDConfig], T] + + +class ConfigBuilder: # pylint: disable=too-few-public-methods + """ + Builder for the data system configuration. + """ + + def __init__(self) -> None: + self._initializers: Optional[List[Builder[Initializer]]] = None + self._primary_synchronizer: Optional[Builder[Synchronizer]] = None + self._secondary_synchronizer: Optional[Builder[Synchronizer]] = None + self._fdv1_fallback_synchronizer: Optional[Builder[Synchronizer]] = None + self._store_mode: DataStoreMode = DataStoreMode.READ_ONLY + self._data_store: Optional[FeatureStore] = None + + def initializers(self, initializers: Optional[List[Builder[Initializer]]]) -> "ConfigBuilder": + """ + Sets the initializers for the data system. + """ + self._initializers = initializers + return self + + def synchronizers( + self, + primary: Builder[Synchronizer], + secondary: Optional[Builder[Synchronizer]] = None, + ) -> "ConfigBuilder": + """ + Sets the synchronizers for the data system. + """ + self._primary_synchronizer = primary + self._secondary_synchronizer = secondary + return self + + def fdv1_compatible_synchronizer( + self, + fallback: Builder[Synchronizer] + ) -> "ConfigBuilder": + """ + Configures the SDK with a fallback synchronizer that is compatible with + the Flag Delivery v1 API. + """ + self._fdv1_fallback_synchronizer = fallback + return self + + def data_store(self, data_store: FeatureStore, store_mode: DataStoreMode) -> "ConfigBuilder": + """ + Sets the data store configuration for the data system. + """ + self._data_store = data_store + self._store_mode = store_mode + return self + + def build(self) -> DataSystemConfig: + """ + Builds the data system configuration. + """ + if self._secondary_synchronizer is not None and self._primary_synchronizer is None: + raise ValueError("Primary synchronizer must be set if secondary is set") + + return DataSystemConfig( + initializers=self._initializers, + primary_synchronizer=self._primary_synchronizer, + secondary_synchronizer=self._secondary_synchronizer, + fdv1_fallback_synchronizer=self._fdv1_fallback_synchronizer, + data_store_mode=self._store_mode, + data_store=self._data_store, + ) + + +def polling_ds_builder() -> Builder[PollingDataSource]: + def builder(config: LDConfig) -> PollingDataSource: + requester = Urllib3PollingRequester(config) + polling_ds = PollingDataSourceBuilder(config) + polling_ds.requester(requester) + + return polling_ds.build() + + return builder + + +def fdv1_fallback_ds_builder() -> Builder[PollingDataSource]: + def builder(config: LDConfig) -> PollingDataSource: + requester = Urllib3FDv1PollingRequester(config) + polling_ds = PollingDataSourceBuilder(config) + polling_ds.requester(requester) + + return polling_ds.build() + + return builder + + +def streaming_ds_builder() -> Builder[StreamingDataSource]: + def builder(config: LDConfig) -> StreamingDataSource: + return StreamingDataSourceBuilder(config).build() + + return builder + + +def default() -> ConfigBuilder: + """ + Default is LaunchDarkly's recommended flag data acquisition strategy. + + Currently, it operates a two-phase method for obtaining data: first, it + requests data from LaunchDarkly's global CDN. Then, it initiates a + streaming connection to LaunchDarkly's Flag Delivery services to + receive real-time updates. + + If the streaming connection is interrupted for an extended period of + time, the SDK will automatically fall back to polling the global CDN + for updates. + """ + + polling_builder = polling_ds_builder() + streaming_builder = streaming_ds_builder() + fallback = fdv1_fallback_ds_builder() + + builder = ConfigBuilder() + builder.initializers([polling_builder]) + builder.synchronizers(streaming_builder, polling_builder) + builder.fdv1_compatible_synchronizer(fallback) + + return builder + + +def streaming() -> ConfigBuilder: + """ + Streaming configures the SDK to efficiently streams flag/segment data + in the background, allowing evaluations to operate on the latest data + with no additional latency. + """ + + streaming_builder = streaming_ds_builder() + fallback = fdv1_fallback_ds_builder() + + builder = ConfigBuilder() + builder.synchronizers(streaming_builder) + builder.fdv1_compatible_synchronizer(fallback) + + return builder + + +def polling() -> ConfigBuilder: + """ + Polling configures the SDK to regularly poll an endpoint for + flag/segment data in the background. This is less efficient than + streaming, but may be necessary in some network environments. + """ + + polling_builder: Builder[Synchronizer] = polling_ds_builder() + fallback = fdv1_fallback_ds_builder() + + builder = ConfigBuilder() + builder.synchronizers(polling_builder) + builder.fdv1_compatible_synchronizer(fallback) + + return builder + + +def custom() -> ConfigBuilder: + """ + Custom returns a builder suitable for creating a custom data + acquisition strategy. You may configure how the SDK uses a Persistent + Store, how the SDK obtains an initial set of data, and how the SDK + keeps data up-to-date. + """ + + return ConfigBuilder() + + +def daemon(store: FeatureStore) -> ConfigBuilder: + """ + Daemon configures the SDK to read from a persistent store integration + that is populated by Relay Proxy or other SDKs. The SDK will not connect + to LaunchDarkly. In this mode, the SDK never writes to the data store. + """ + return default().data_store(store, DataStoreMode.READ_ONLY) + + +def persistent_store(store: FeatureStore) -> ConfigBuilder: + """ + PersistentStore is similar to Default, with the addition of a persistent + store integration. Before data has arrived from LaunchDarkly, the SDK is + able to evaluate flags using data from the persistent store. Once fresh + data is available, the SDK will no longer read from the persistent store, + although it will keep it up-to-date. + """ + return default().data_store(store, DataStoreMode.READ_WRITE) diff --git a/ldclient/impl/datasourcev2/__init__.py b/ldclient/impl/datasourcev2/__init__.py index 1bde435b..f650e9a4 100644 --- a/ldclient/impl/datasourcev2/__init__.py +++ b/ldclient/impl/datasourcev2/__init__.py @@ -1,6 +1,6 @@ """ -This module houses FDv2 types and implementations of synchronizers and -initializers for the datasystem. +This module houses FDv2 implementations of synchronizers and initializers for +the datasystem. All types and implementations in this module are considered internal and are not part of the public API of the LaunchDarkly Python SDK. @@ -9,7 +9,3 @@ You have been warned. """ - -from .polling import PollingResult, Requester - -__all__: list[str] = ["PollingResult", "Requester"] diff --git a/ldclient/impl/datasourcev2/polling.py b/ldclient/impl/datasourcev2/polling.py index 224f49c5..eba635a4 100644 --- a/ldclient/impl/datasourcev2/polling.py +++ b/ldclient/impl/datasourcev2/polling.py @@ -14,21 +14,17 @@ import urllib3 from ldclient.config import Config -from ldclient.impl.datasystem import BasisResult, Update +from ldclient.impl.datasource.feature_requester import LATEST_ALL_URI from ldclient.impl.datasystem.protocolv2 import ( - Basis, - ChangeSet, - ChangeSetBuilder, DeleteObject, EventName, - IntentCode, - PutObject, - Selector, - ServerIntent + PutObject ) from ldclient.impl.http import _http_factory from ldclient.impl.repeating_task import RepeatingTask from ldclient.impl.util import ( + _LD_ENVID_HEADER, + _LD_FD_FALLBACK_HEADER, UnsuccessfulResponseException, _Fail, _headers, @@ -39,9 +35,21 @@ log ) from ldclient.interfaces import ( + Basis, + BasisResult, + ChangeSet, + ChangeSetBuilder, DataSourceErrorInfo, DataSourceErrorKind, - DataSourceState + DataSourceState, + Initializer, + IntentCode, + ObjectKind, + Selector, + SelectorStore, + ServerIntent, + Synchronizer, + Update ) POLLING_ENDPOINT = "/sdk/poll" @@ -72,7 +80,7 @@ def fetch(self, selector: Optional[Selector]) -> PollingResult: CacheEntry = namedtuple("CacheEntry", ["data", "etag"]) -class PollingDataSource: +class PollingDataSource(Initializer, Synchronizer): """ PollingDataSource is a data source that can retrieve information from LaunchDarkly either as an Initializer or as a Synchronizer. @@ -86,30 +94,40 @@ def __init__( self._requester = requester self._poll_interval = poll_interval self._event = Event() + self._stop = Event() self._task = RepeatingTask( "ldclient.datasource.polling", poll_interval, 0, self._poll ) + @property def name(self) -> str: """Returns the name of the initializer.""" return "PollingDataSourceV2" - def fetch(self) -> BasisResult: + def fetch(self, ss: SelectorStore) -> BasisResult: """ Fetch returns a Basis, or an error if the Basis could not be retrieved. """ - return self._poll() + return self._poll(ss) - def sync(self) -> Generator[Update, None, None]: + def sync(self, ss: SelectorStore) -> Generator[Update, None, None]: """ sync begins the synchronization process for the data source, yielding Update objects until the connection is closed or an unrecoverable error occurs. """ log.info("Starting PollingDataSourceV2 synchronizer") - while True: - result = self._requester.fetch(None) + self._stop.clear() + while self._stop.is_set() is False: + result = self._requester.fetch(ss.selector()) if isinstance(result, _Fail): + fallback = None + envid = None + + if result.headers is not None: + fallback = result.headers.get(_LD_FD_FALLBACK_HEADER) == 'true' + envid = result.headers.get(_LD_ENVID_HEADER) + if isinstance(result.exception, UnsuccessfulResponseException): error_info = DataSourceErrorInfo( kind=DataSourceErrorKind.ERROR_RESPONSE, @@ -120,19 +138,28 @@ def sync(self) -> Generator[Update, None, None]: ), ) + if fallback: + yield Update( + state=DataSourceState.OFF, + error=error_info, + revert_to_fdv1=True, + environment_id=envid, + ) + break + status_code = result.exception.status if is_http_error_recoverable(status_code): - # TODO(fdv2): Add support for environment ID yield Update( state=DataSourceState.INTERRUPTED, error=error_info, + environment_id=envid, ) continue - # TODO(fdv2): Add support for environment ID yield Update( state=DataSourceState.OFF, error=error_info, + environment_id=envid, ) break @@ -143,27 +170,33 @@ def sync(self) -> Generator[Update, None, None]: message=result.error, ) - # TODO(fdv2): Go has a designation here to handle JSON decoding separately. - # TODO(fdv2): Add support for environment ID yield Update( state=DataSourceState.INTERRUPTED, error=error_info, + environment_id=envid, ) else: (change_set, headers) = result.value yield Update( state=DataSourceState.VALID, change_set=change_set, - environment_id=headers.get("X-LD-EnvID"), + environment_id=headers.get(_LD_ENVID_HEADER), + revert_to_fdv1=headers.get(_LD_FD_FALLBACK_HEADER) == 'true' ) if self._event.wait(self._poll_interval): break - def _poll(self) -> BasisResult: + def stop(self): + """Stops the synchronizer.""" + log.info("Stopping PollingDataSourceV2 synchronizer") + self._event.set() + self._task.stop() + self._stop.set() + + def _poll(self, ss: SelectorStore) -> BasisResult: try: - # TODO(fdv2): Need to pass the selector through - result = self._requester.fetch(None) + result = self._requester.fetch(ss.selector()) if isinstance(result, _Fail): if isinstance(result.exception, UnsuccessfulResponseException): @@ -185,7 +218,7 @@ def _poll(self) -> BasisResult: (change_set, headers) = result.value - env_id = headers.get("X-LD-EnvID") + env_id = headers.get(_LD_ENVID_HEADER) if not isinstance(env_id, str): env_id = None @@ -204,7 +237,7 @@ def _poll(self) -> BasisResult: # pylint: disable=too-few-public-methods -class Urllib3PollingRequester: +class Urllib3PollingRequester(Requester): """ Urllib3PollingRequester is a Requester that uses urllib3 to make HTTP requests. @@ -226,7 +259,7 @@ def fetch(self, selector: Optional[Selector]) -> PollingResult: if self._config.payload_filter_key is not None: query_params["filter"] = self._config.payload_filter_key - if selector is not None: + if selector is not None and selector.is_defined(): query_params["selector"] = selector.state uri = self._poll_uri @@ -250,14 +283,14 @@ def fetch(self, selector: Optional[Selector]) -> PollingResult: ), retries=1, ) + headers = response.headers if response.status >= 400: return _Fail( - f"HTTP error {response}", UnsuccessfulResponseException(response.status) + f"HTTP error {response}", UnsuccessfulResponseException(response.status), + headers=headers, ) - headers = response.headers - if response.status == 304: return _Success(value=(ChangeSetBuilder.no_changes(), headers)) @@ -281,6 +314,7 @@ def fetch(self, selector: Optional[Selector]) -> PollingResult: return _Fail( error=changeset_result.error, exception=changeset_result.exception, + headers=headers, # type: ignore ) @@ -366,3 +400,119 @@ def build(self) -> PollingDataSource: return PollingDataSource( poll_interval=self._config.poll_interval, requester=requester ) + + +# pylint: disable=too-few-public-methods +class Urllib3FDv1PollingRequester(Requester): + """ + Urllib3PollingRequesterFDv1 is a Requester that uses urllib3 to make HTTP + requests. + """ + + def __init__(self, config: Config): + self._etag = None + self._http = _http_factory(config).create_pool_manager(1, config.base_uri) + self._config = config + self._poll_uri = config.base_uri + LATEST_ALL_URI + + def fetch(self, selector: Optional[Selector]) -> PollingResult: + """ + Fetches the data for the given selector. + Returns a Result containing a tuple of ChangeSet and any request headers, + or an error if the data could not be retrieved. + """ + query_params = {} + if self._config.payload_filter_key is not None: + query_params["filter"] = self._config.payload_filter_key + + uri = self._poll_uri + if len(query_params) > 0: + filter_query = parse.urlencode(query_params) + uri += f"?{filter_query}" + + hdrs = _headers(self._config) + hdrs["Accept-Encoding"] = "gzip" + + if self._etag is not None: + hdrs["If-None-Match"] = self._etag + + response = self._http.request( + "GET", + uri, + headers=hdrs, + timeout=urllib3.Timeout( + connect=self._config.http.connect_timeout, + read=self._config.http.read_timeout, + ), + retries=1, + ) + + headers = response.headers + if response.status >= 400: + return _Fail( + f"HTTP error {response}", UnsuccessfulResponseException(response.status), + headers=headers + ) + + if response.status == 304: + return _Success(value=(ChangeSetBuilder.no_changes(), headers)) + + data = json.loads(response.data.decode("UTF-8")) + etag = headers.get("ETag") + + if etag is not None: + self._etag = etag + + log.debug( + "%s response status:[%d] ETag:[%s]", + uri, + response.status, + etag, + ) + + changeset_result = fdv1_polling_payload_to_changeset(data) + if isinstance(changeset_result, _Success): + return _Success(value=(changeset_result.value, headers)) + + return _Fail( + error=changeset_result.error, + exception=changeset_result.exception, + headers=headers, + ) + + +# pylint: disable=too-many-branches,too-many-return-statements +def fdv1_polling_payload_to_changeset(data: dict) -> _Result[ChangeSet, str]: + """ + Converts a fdv1 polling payload into a ChangeSet. + """ + builder = ChangeSetBuilder() + builder.start(IntentCode.TRANSFER_FULL) + selector = Selector.no_selector() + + # FDv1 uses "flags" instead of "features", so we need to map accordingly + # Map FDv1 JSON keys to ObjectKind enum values + kind_mappings = [ + (ObjectKind.FLAG, "flags"), + (ObjectKind.SEGMENT, "segments") + ] + + for kind, fdv1_key in kind_mappings: + kind_data = data.get(fdv1_key) + if kind_data is None: + continue + if not isinstance(kind_data, dict): + return _Fail(error=f"Invalid format: {fdv1_key} is not a dictionary") + + for key in kind_data: + flag_or_segment = kind_data.get(key) + if flag_or_segment is None or not isinstance(flag_or_segment, dict): + return _Fail(error=f"Invalid format: {key} is not a dictionary") + + version = flag_or_segment.get('version') + if version is None: + return _Fail(error=f"Invalid format: {key} does not have a version set") + + builder.add_put(kind, key, version, flag_or_segment) + + return _Success(builder.finish(selector)) diff --git a/ldclient/impl/datasourcev2/streaming.py b/ldclient/impl/datasourcev2/streaming.py index 03ea68ff..d79a341d 100644 --- a/ldclient/impl/datasourcev2/streaming.py +++ b/ldclient/impl/datasourcev2/streaming.py @@ -4,13 +4,12 @@ """ import json -from abc import abstractmethod from time import time -from typing import Callable, Generator, Iterable, Optional, Protocol, Tuple +from typing import Callable, Generator, Optional, Tuple from urllib import parse -from ld_eventsource import SSEClient as SSEClientImpl -from ld_eventsource.actions import Action, Event, Fault +from ld_eventsource import SSEClient +from ld_eventsource.actions import Event, Fault, Start from ld_eventsource.config import ( ConnectStrategy, ErrorStrategy, @@ -19,28 +18,33 @@ from ld_eventsource.errors import HTTPStatusError from ldclient.config import Config -from ldclient.impl.datasystem import Synchronizer, Update +from ldclient.impl.datasystem import DiagnosticAccumulator, DiagnosticSource from ldclient.impl.datasystem.protocolv2 import ( - ChangeSetBuilder, DeleteObject, Error, EventName, Goodbye, - IntentCode, - PutObject, - Selector, - ServerIntent + PutObject ) from ldclient.impl.http import HTTPFactory, _http_factory from ldclient.impl.util import ( + _LD_ENVID_HEADER, + _LD_FD_FALLBACK_HEADER, http_error_message, is_http_error_recoverable, log ) from ldclient.interfaces import ( + ChangeSetBuilder, DataSourceErrorInfo, DataSourceErrorKind, - DataSourceState + DataSourceState, + IntentCode, + Selector, + SelectorStore, + ServerIntent, + Synchronizer, + Update ) # allows for up to 5 minutes to elapse without any data sent across the stream. @@ -53,34 +57,17 @@ STREAMING_ENDPOINT = "/sdk/stream" - -class SSEClient(Protocol): # pylint: disable=too-few-public-methods - """ - SSEClient is a protocol that defines the interface for a client that can - connect to a Server-Sent Events (SSE) stream and provide an iterable of - actions received from that stream. - """ - - @property - @abstractmethod - def all(self) -> Iterable[Action]: - """ - Returns an iterable of all actions received from the SSE stream. - """ - raise NotImplementedError - - -SseClientBuilder = Callable[[Config], SSEClient] +SseClientBuilder = Callable[[Config, SelectorStore], SSEClient] -# TODO(sdk-1391): Pass a selector-retrieving function through so it can -# re-connect with the last known status. -def create_sse_client(config: Config) -> SSEClientImpl: +def create_sse_client(config: Config, ss: SelectorStore) -> SSEClient: """ " - create_sse_client creates an SSEClientImpl instance configured to connect + create_sse_client creates an SSEClient instance configured to connect to the LaunchDarkly streaming endpoint. """ uri = config.stream_base_uri + STREAMING_ENDPOINT + if config.payload_filter_key is not None: + uri += "?%s" % parse.urlencode({"filter": config.payload_filter_key}) # We don't want the stream to use the same read timeout as the rest of the SDK. http_factory = _http_factory(config) @@ -90,12 +77,17 @@ def create_sse_client(config: Config) -> SSEClientImpl: override_read_timeout=STREAM_READ_TIMEOUT, ) - return SSEClientImpl( + def query_params() -> dict[str, str]: + selector = ss.selector() + return {"basis": selector.state} if selector.is_defined() else {} + + return SSEClient( connect=ConnectStrategy.http( url=uri, headers=http_factory.base_headers, pool=stream_http_factory.create_pool_manager(1, uri), urllib3_request_options={"timeout": stream_http_factory.timeout}, + query_params=query_params ), # we'll make error-handling decisions when we see a Fault error_strategy=ErrorStrategy.always_continue(), @@ -110,7 +102,7 @@ def create_sse_client(config: Config) -> SSEClientImpl: ) -class StreamingDataSource(Synchronizer): +class StreamingDataSource(Synchronizer, DiagnosticSource): """ StreamingSynchronizer is a specific type of Synchronizer that handles streaming data sources. @@ -119,30 +111,40 @@ class StreamingDataSource(Synchronizer): from the streaming data source. """ - def __init__( - self, config: Config, sse_client_builder: SseClientBuilder = create_sse_client - ): - self._sse_client_builder = sse_client_builder - self._uri = config.stream_base_uri + STREAMING_ENDPOINT - if config.payload_filter_key is not None: - self._uri += "?%s" % parse.urlencode({"filter": config.payload_filter_key}) + def __init__(self, config: Config): + self._sse_client_builder = create_sse_client self._config = config self._sse: Optional[SSEClient] = None + self._running = False + self._diagnostic_accumulator: Optional[DiagnosticAccumulator] = None + self._connection_attempt_start_time: Optional[float] = None + + def set_diagnostic_accumulator(self, diagnostic_accumulator: DiagnosticAccumulator): + self._diagnostic_accumulator = diagnostic_accumulator - def sync(self) -> Generator[Update, None, None]: + @property + def name(self) -> str: + """ + Returns the name of the synchronizer, which is used for logging and debugging. + """ + return "streaming" + + def sync(self, ss: SelectorStore) -> Generator[Update, None, None]: """ sync should begin the synchronization process for the data source, yielding Update objects until the connection is closed or an unrecoverable error occurs. """ - log.info("Starting StreamingUpdateProcessor connecting to uri: %s", self._uri) - self._sse = self._sse_client_builder(self._config) + self._sse = self._sse_client_builder(self._config, ss) if self._sse is None: log.error("Failed to create SSE client for streaming updates.") return change_set_builder = ChangeSetBuilder() + self._running = True + self._connection_attempt_start_time = time() + envid = None for action in self._sse.all: if isinstance(action, Fault): # If the SSE client detects the stream has closed, then it will @@ -151,7 +153,10 @@ def sync(self) -> Generator[Update, None, None]: if action.error is None: continue - (update, should_continue) = self._handle_error(action.error) + if action.headers is not None: + envid = action.headers.get(_LD_ENVID_HEADER, envid) + + (update, should_continue) = self._handle_error(action.error, envid) if update is not None: yield update @@ -159,21 +164,35 @@ def sync(self) -> Generator[Update, None, None]: break continue + if isinstance(action, Start) and action.headers is not None: + fallback = action.headers.get(_LD_FD_FALLBACK_HEADER) == 'true' + envid = action.headers.get(_LD_ENVID_HEADER, envid) + + if fallback: + self._record_stream_init(True) + yield Update( + state=DataSourceState.OFF, + revert_to_fdv1=True, + environment_id=envid, + ) + break + if not isinstance(action, Event): continue try: - update = self._process_message(action, change_set_builder) + update = self._process_message(action, change_set_builder, envid) if update is not None: + self._record_stream_init(False) + self._connection_attempt_start_time = None yield update except json.decoder.JSONDecodeError as e: log.info( "Error while handling stream event; will restart stream: %s", e ) - # TODO(sdk-1409) - # self._sse.interrupt() + self._sse.interrupt() - (update, should_continue) = self._handle_error(e) + (update, should_continue) = self._handle_error(e, envid) if update is not None: yield update if not should_continue: @@ -182,8 +201,7 @@ def sync(self) -> Generator[Update, None, None]: log.info( "Error while handling stream event; will restart stream: %s", e ) - # TODO(sdk-1409) - # self._sse.interrupt() + self._sse.interrupt() yield Update( state=DataSourceState.INTERRUPTED, @@ -191,43 +209,29 @@ def sync(self) -> Generator[Update, None, None]: DataSourceErrorKind.UNKNOWN, 0, time(), str(e) ), revert_to_fdv1=False, - environment_id=None, # TODO(sdk-1410) + environment_id=envid, ) - # TODO(sdk-1408) - # if update is not None: - # self._record_stream_init(False) - - # if self._data_source_update_sink is not None: - # self._data_source_update_sink.update_status( - # DataSourceState.VALID, None - # ) - - # if not self._ready.is_set(): - # log.info("StreamingUpdateProcessor initialized ok.") - # self._ready.set() - - # TODO(sdk-1409) - # self._sse.close() - - # TODO(sdk-1409) - # def stop(self): - # self.__stop_with_error_info(None) - # - # def __stop_with_error_info(self, error: Optional[DataSourceErrorInfo]): - # log.info("Stopping StreamingUpdateProcessor") - # self._running = False - # if self._sse: - # self._sse.close() - # - # if self._data_source_update_sink is None: - # return - # - # self._data_source_update_sink.update_status(DataSourceState.OFF, error) + self._sse.close() + + def stop(self): + """ + Stops the streaming synchronizer, closing any open connections. + """ + log.info("Stopping StreamingUpdateProcessor") + self._running = False + if self._sse: + self._sse.close() + + def _record_stream_init(self, failed: bool): + if self._diagnostic_accumulator and self._connection_attempt_start_time: + current_time = int(time() * 1000) + elapsed = current_time - int(self._connection_attempt_start_time * 1000) + self._diagnostic_accumulator.record_stream_init(current_time, elapsed if elapsed >= 0 else 0, failed) # pylint: disable=too-many-return-statements def _process_message( - self, msg: Event, change_set_builder: ChangeSetBuilder + self, msg: Event, change_set_builder: ChangeSetBuilder, envid: Optional[str] ) -> Optional[Update]: """ Processes a single message from the SSE stream and returns an Update @@ -248,7 +252,7 @@ def _process_message( change_set_builder.expect_changes() return Update( state=DataSourceState.VALID, - environment_id=None, # TODO(sdk-1410) + environment_id=envid, ) return None @@ -294,13 +298,13 @@ def _process_message( return Update( state=DataSourceState.VALID, change_set=change_set, - environment_id=None, # TODO(sdk-1410) + environment_id=envid, ) log.info("Unexpected event found in stream: %s", msg.event) return None - def _handle_error(self, error: Exception) -> Tuple[Optional[Update], bool]: + def _handle_error(self, error: Exception, envid: Optional[str]) -> Tuple[Optional[Update], bool]: """ This method handles errors that occur during the streaming process. @@ -309,14 +313,19 @@ def _handle_error(self, error: Exception) -> Tuple[Optional[Update], bool]: If an update is provided, it should be forward upstream, regardless of whether or not we are going to retry this failure. + + The return should be thought of (update, should_continue) """ - # if not self._running: - # return (False, None) # don't retry if we've been deliberately stopped + if not self._running: + return (None, False) # don't retry if we've been deliberately stopped update: Optional[Update] = None if isinstance(error, json.decoder.JSONDecodeError): log.error("Unexpected error on stream connection: %s, will retry", error) + self._record_stream_init(True) + self._connection_attempt_start_time = time() + \ + self._sse.next_retry_delay # type: ignore update = Update( state=DataSourceState.INTERRUPTED, @@ -324,11 +333,15 @@ def _handle_error(self, error: Exception) -> Tuple[Optional[Update], bool]: DataSourceErrorKind.INVALID_DATA, 0, time(), str(error) ), revert_to_fdv1=False, - environment_id=None, # TODO(sdk-1410) + environment_id=envid, ) return (update, True) if isinstance(error, HTTPStatusError): + self._record_stream_init(True) + self._connection_attempt_start_time = time() + \ + self._sse.next_retry_delay # type: ignore + error_info = DataSourceErrorInfo( DataSourceErrorKind.ERROR_RESPONSE, error.status, @@ -336,12 +349,22 @@ def _handle_error(self, error: Exception) -> Tuple[Optional[Update], bool]: str(error), ) + if envid is None and error.headers is not None: + envid = error.headers.get(_LD_ENVID_HEADER) + + if error.headers is not None and error.headers.get(_LD_FD_FALLBACK_HEADER) == 'true': + update = Update( + state=DataSourceState.OFF, + error=error_info, + revert_to_fdv1=True, + environment_id=envid, + ) + return (update, False) + http_error_message_result = http_error_message( error.status, "stream connection" ) - is_recoverable = is_http_error_recoverable(error.status) - update = Update( state=( DataSourceState.INTERRUPTED @@ -350,21 +373,21 @@ def _handle_error(self, error: Exception) -> Tuple[Optional[Update], bool]: ), error=error_info, revert_to_fdv1=False, - environment_id=None, # TODO(sdk-1410) + environment_id=envid, ) if not is_recoverable: + self._connection_attempt_start_time = None log.error(http_error_message_result) - # TODO(sdk-1409) - # self._ready.set() # if client is initializing, make it stop waiting; has no effect if already inited - # self.__stop_with_error_info(error_info) - # self.stop() + self.stop() return (update, False) log.warning(http_error_message_result) return (update, True) log.warning("Unexpected error on stream connection: %s, will retry", error) + self._record_stream_init(True) + self._connection_attempt_start_time = time() + self._sse.next_retry_delay # type: ignore update = Update( state=DataSourceState.INTERRUPTED, @@ -372,21 +395,13 @@ def _handle_error(self, error: Exception) -> Tuple[Optional[Update], bool]: DataSourceErrorKind.UNKNOWN, 0, time(), str(error) ), revert_to_fdv1=False, - environment_id=None, # TODO(sdk-1410) + environment_id=envid, ) # no stacktrace here because, for a typical connection error, it'll # just be a lengthy tour of urllib3 internals return (update, True) - # magic methods for "with" statement (used in testing) - def __enter__(self): - return self - - def __exit__(self, type, value, traceback): - # self.stop() - pass - class StreamingDataSourceBuilder: # disable: pylint: disable=too-few-public-methods """ @@ -398,5 +413,4 @@ def __init__(self, config: Config): def build(self) -> StreamingDataSource: """Builds a StreamingDataSource instance with the configured parameters.""" - # TODO(fdv2): Add in the other controls here. return StreamingDataSource(self._config) diff --git a/ldclient/impl/datastore/status.py b/ldclient/impl/datastore/status.py index a8dd5ee3..ee9797dd 100644 --- a/ldclient/impl/datastore/status.py +++ b/ldclient/impl/datastore/status.py @@ -1,7 +1,7 @@ from __future__ import annotations from copy import copy -from typing import TYPE_CHECKING, Callable +from typing import TYPE_CHECKING, Callable, Protocol from ldclient.impl.listeners import Listeners from ldclient.impl.rwlock import ReadWriteLock diff --git a/ldclient/impl/datasystem/__init__.py b/ldclient/impl/datasystem/__init__.py index 9c5bf6d6..c7a36829 100644 --- a/ldclient/impl/datasystem/__init__.py +++ b/ldclient/impl/datasystem/__init__.py @@ -4,19 +4,15 @@ """ from abc import abstractmethod -from dataclasses import dataclass from enum import Enum from threading import Event -from typing import Generator, Optional, Protocol +from typing import Protocol, runtime_checkable -from ldclient.impl.datasystem.protocolv2 import Basis, ChangeSet -from ldclient.impl.util import _Result from ldclient.interfaces import ( - DataSourceErrorInfo, - DataSourceState, DataSourceStatusProvider, DataStoreStatusProvider, - FlagTracker + FlagTracker, + ReadOnlyStore ) @@ -141,59 +137,31 @@ def target_availability(self) -> DataAvailability: """ raise NotImplementedError - -BasisResult = _Result[Basis, str] - - -class Initializer(Protocol): # pylint: disable=too-few-public-methods - """ - Initializer represents a component capable of retrieving a single data - result, such as from the LD polling API. - - The intent of initializers is to quickly fetch an initial set of data, - which may be stale but is fast to retrieve. This initial data serves as a - foundation for a Synchronizer to build upon, enabling it to provide updates - as new changes occur. - """ - + @property @abstractmethod - def fetch(self) -> BasisResult: + def store(self) -> ReadOnlyStore: """ - fetch should retrieve the initial data set for the data source, returning - a Basis object on success, or an error message on failure. + Returns the data store used by the data system. """ raise NotImplementedError -@dataclass(frozen=True) -class Update: - """ - Update represents the results of a synchronizer's ongoing sync - method. - """ - - state: DataSourceState - change_set: Optional[ChangeSet] = None - error: Optional[DataSourceErrorInfo] = None - revert_to_fdv1: bool = False - environment_id: Optional[str] = None +class DiagnosticAccumulator(Protocol): + def record_stream_init(self, timestamp, duration, failed): + raise NotImplementedError + def record_events_in_batch(self, events_in_batch): + raise NotImplementedError -class Synchronizer(Protocol): # pylint: disable=too-few-public-methods - """ - Synchronizer represents a component capable of synchronizing data from an external - data source, such as a streaming or polling API. + def create_event_and_reset(self, dropped_events, deduplicated_users): + raise NotImplementedError - It is responsible for yielding Update objects that represent the current state - of the data source, including any changes that have occurred since the last - synchronization. - """ +@runtime_checkable +class DiagnosticSource(Protocol): @abstractmethod - def sync(self) -> Generator[Update, None, None]: + def set_diagnostic_accumulator(self, diagnostic_accumulator: DiagnosticAccumulator): """ - sync should begin the synchronization process for the data source, yielding - Update objects until the connection is closed or an unrecoverable error - occurs. + Set the diagnostic_accumulator to be used for reporting diagnostic events. """ raise NotImplementedError diff --git a/ldclient/impl/datasystem/config.py b/ldclient/impl/datasystem/config.py deleted file mode 100644 index c0e66d6b..00000000 --- a/ldclient/impl/datasystem/config.py +++ /dev/null @@ -1,188 +0,0 @@ -""" -Configuration for LaunchDarkly's data acquisition strategy. -""" - -from dataclasses import dataclass -from typing import Callable, List, Optional, TypeVar - -from ldclient.config import Config as LDConfig -from ldclient.impl.datasourcev2.polling import ( - PollingDataSource, - PollingDataSourceBuilder, - Urllib3PollingRequester -) -from ldclient.impl.datasourcev2.streaming import ( - StreamingDataSource, - StreamingDataSourceBuilder -) -from ldclient.impl.datasystem import Initializer, Synchronizer - -T = TypeVar("T") - -Builder = Callable[[], T] - - -@dataclass(frozen=True) -class Config: - """ - Configuration for LaunchDarkly's data acquisition strategy. - """ - - initializers: Optional[List[Builder[Initializer]]] - """The initializers for the data system.""" - - primary_synchronizer: Builder[Synchronizer] - """The primary synchronizer for the data system.""" - - secondary_synchronizer: Optional[Builder[Synchronizer]] - """The secondary synchronizers for the data system.""" - - -class ConfigBuilder: # pylint: disable=too-few-public-methods - """ - Builder for the data system configuration. - """ - - _initializers: Optional[List[Builder[Initializer]]] = None - _primary_synchronizer: Optional[Builder[Synchronizer]] = None - _secondary_synchronizer: Optional[Builder[Synchronizer]] = None - - def initializers(self, initializers: List[Builder[Initializer]]) -> "ConfigBuilder": - """ - Sets the initializers for the data system. - """ - self._initializers = initializers - return self - - def synchronizers( - self, - primary: Builder[Synchronizer], - secondary: Optional[Builder[Synchronizer]] = None, - ) -> "ConfigBuilder": - """ - Sets the synchronizers for the data system. - """ - self._primary_synchronizer = primary - self._secondary_synchronizer = secondary - return self - - def build(self) -> Config: - """ - Builds the data system configuration. - """ - if self._primary_synchronizer is None: - raise ValueError("Primary synchronizer must be set") - - return Config( - initializers=self._initializers, - primary_synchronizer=self._primary_synchronizer, - secondary_synchronizer=self._secondary_synchronizer, - ) - - -def __polling_ds_builder(config: LDConfig) -> Builder[PollingDataSource]: - def builder() -> PollingDataSource: - requester = Urllib3PollingRequester(config) - polling_ds = PollingDataSourceBuilder(config) - polling_ds.requester(requester) - - return polling_ds.build() - - return builder - - -def __streaming_ds_builder(config: LDConfig) -> Builder[StreamingDataSource]: - def builder() -> StreamingDataSource: - return StreamingDataSourceBuilder(config).build() - - return builder - - -def default(config: LDConfig) -> ConfigBuilder: - """ - Default is LaunchDarkly's recommended flag data acquisition strategy. - - Currently, it operates a two-phase method for obtaining data: first, it - requests data from LaunchDarkly's global CDN. Then, it initiates a - streaming connection to LaunchDarkly's Flag Delivery services to - receive real-time updates. - - If the streaming connection is interrupted for an extended period of - time, the SDK will automatically fall back to polling the global CDN - for updates. - """ - - polling_builder = __polling_ds_builder(config) - streaming_builder = __streaming_ds_builder(config) - - builder = ConfigBuilder() - builder.initializers([polling_builder]) - builder.synchronizers(streaming_builder, polling_builder) - - return builder - - -def streaming(config: LDConfig) -> ConfigBuilder: - """ - Streaming configures the SDK to efficiently streams flag/segment data - in the background, allowing evaluations to operate on the latest data - with no additional latency. - """ - - streaming_builder = __streaming_ds_builder(config) - - builder = ConfigBuilder() - builder.synchronizers(streaming_builder) - - return builder - - -def polling(config: LDConfig) -> ConfigBuilder: - """ - Polling configures the SDK to regularly poll an endpoint for - flag/segment data in the background. This is less efficient than - streaming, but may be necessary in some network environments. - """ - - polling_builder = __polling_ds_builder(config) - - builder = ConfigBuilder() - builder.synchronizers(polling_builder) - - return builder - - -def custom() -> ConfigBuilder: - """ - Custom returns a builder suitable for creating a custom data - acquisition strategy. You may configure how the SDK uses a Persistent - Store, how the SDK obtains an initial set of data, and how the SDK - keeps data up-to-date. - """ - - return ConfigBuilder() - - -# TODO(fdv2): Implement these methods -# -# Daemon configures the SDK to read from a persistent store integration -# that is populated by Relay Proxy or other SDKs. The SDK will not connect -# to LaunchDarkly. In this mode, the SDK never writes to the data store. - -# PersistentStore is similar to Default, with the addition of a persistent -# store integration. Before data has arrived from LaunchDarkly, the SDK is -# able to evaluate flags using data from the persistent store. Once fresh -# data is available, the SDK will no longer read from the persistent store, -# although it will keep it up-to-date. - -# WithEndpoints configures the data system with custom endpoints for -# LaunchDarkly's streaming and polling synchronizers. This method is not -# necessary for most use-cases, but can be useful for testing or custom -# network configurations. -# -# Any endpoint that is not specified (empty string) will be treated as the -# default LaunchDarkly SaaS endpoint for that service. - -# WithRelayProxyEndpoints configures the data system with a single endpoint -# for LaunchDarkly's streaming and polling synchronizers. The endpoint -# should be Relay Proxy's base URI, for example http://localhost:8123. diff --git a/ldclient/impl/datasystem/fdv1.py b/ldclient/impl/datasystem/fdv1.py index d291aba3..32af49d6 100644 --- a/ldclient/impl/datasystem/fdv1.py +++ b/ldclient/impl/datasystem/fdv1.py @@ -13,7 +13,11 @@ DataStoreStatusProviderImpl, DataStoreUpdateSinkImpl ) -from ldclient.impl.datasystem import DataAvailability +from ldclient.impl.datasystem import ( + DataAvailability, + DataSystem, + DiagnosticAccumulator +) from ldclient.impl.flag_tracker import FlagTrackerImpl from ldclient.impl.listeners import Listeners from ldclient.impl.stubs import NullUpdateProcessor @@ -24,13 +28,14 @@ DataStoreStatusProvider, FeatureStore, FlagTracker, + ReadOnlyStore, UpdateProcessor ) # Delayed import inside __init__ to avoid circular dependency with ldclient.client -class FDv1: +class FDv1(DataSystem): """ FDv1 wires the existing v1 data source and store behavior behind the generic DataSystem surface. @@ -77,7 +82,7 @@ def __init__(self, config: Config): self._update_processor: Optional[UpdateProcessor] = None # Diagnostic accumulator provided by client for streaming metrics - self._diagnostic_accumulator = None + self._diagnostic_accumulator: Optional[DiagnosticAccumulator] = None # Track current data availability self._data_availability: DataAvailability = ( @@ -110,7 +115,7 @@ def stop(self): self._update_processor.stop() @property - def store(self) -> FeatureStore: + def store(self) -> ReadOnlyStore: return self._store_wrapper def set_flag_value_eval_fn(self, eval_fn): @@ -121,7 +126,7 @@ def set_flag_value_eval_fn(self, eval_fn): """ self._flag_tracker_impl = FlagTrackerImpl(self._flag_change_listeners, eval_fn) - def set_diagnostic_accumulator(self, diagnostic_accumulator): + def set_diagnostic_accumulator(self, diagnostic_accumulator: DiagnosticAccumulator): """ Sets the diagnostic accumulator for streaming initialization metrics. This should be called before start() to ensure metrics are collected. @@ -142,7 +147,16 @@ def flag_tracker(self) -> FlagTracker: @property def data_availability(self) -> DataAvailability: - return self._data_availability + if self._config.offline: + return DataAvailability.DEFAULTS + + if self._update_processor is not None and self._update_processor.initialized(): + return DataAvailability.REFRESHED + + if self._store_wrapper.initialized: + return DataAvailability.CACHED + + return DataAvailability.DEFAULTS @property def target_availability(self) -> DataAvailability: diff --git a/ldclient/impl/datasystem/fdv2.py b/ldclient/impl/datasystem/fdv2.py new file mode 100644 index 00000000..21f95c0a --- /dev/null +++ b/ldclient/impl/datasystem/fdv2.py @@ -0,0 +1,701 @@ +import time +from copy import copy +from queue import Queue +from threading import Event, Thread +from typing import Any, Callable, Dict, List, Mapping, Optional + +from ldclient.config import Builder, Config, DataSystemConfig +from ldclient.feature_store import _FeatureStoreDataSetSorter +from ldclient.impl.datasystem import ( + DataAvailability, + DataSystem, + DiagnosticAccumulator, + DiagnosticSource +) +from ldclient.impl.datasystem.store import Store +from ldclient.impl.flag_tracker import FlagTrackerImpl +from ldclient.impl.listeners import Listeners +from ldclient.impl.repeating_task import RepeatingTask +from ldclient.impl.rwlock import ReadWriteLock +from ldclient.impl.util import _Fail, log +from ldclient.interfaces import ( + DataSourceErrorInfo, + DataSourceState, + DataSourceStatus, + DataSourceStatusProvider, + DataStoreMode, + DataStoreStatus, + DataStoreStatusProvider, + FeatureStore, + FlagTracker, + ReadOnlyStore, + Synchronizer +) +from ldclient.versioned_data_kind import VersionedDataKind + + +class DataSourceStatusProviderImpl(DataSourceStatusProvider): + def __init__(self, listeners: Listeners): + self.__listeners = listeners + self.__status = DataSourceStatus(DataSourceState.INITIALIZING, time.time(), None) + self.__lock = ReadWriteLock() + + @property + def status(self) -> DataSourceStatus: + self.__lock.rlock() + status = self.__status + self.__lock.runlock() + + return status + + def update_status(self, new_state: DataSourceState, new_error: Optional[DataSourceErrorInfo]): + status_to_broadcast = None + + try: + self.__lock.lock() + old_status = self.__status + + if new_state == DataSourceState.INTERRUPTED and old_status.state == DataSourceState.INITIALIZING: + new_state = DataSourceState.INITIALIZING + + if new_state == old_status.state and new_error is None: + return + + new_since = self.__status.since if new_state == self.__status.state else time.time() + new_error = self.__status.error if new_error is None else new_error + + self.__status = DataSourceStatus(new_state, new_since, new_error) + + status_to_broadcast = self.__status + finally: + self.__lock.unlock() + + if status_to_broadcast is not None: + self.__listeners.notify(status_to_broadcast) + + def add_listener(self, listener: Callable[[DataSourceStatus], None]): + self.__listeners.add(listener) + + def remove_listener(self, listener: Callable[[DataSourceStatus], None]): + self.__listeners.remove(listener) + + +class DataStoreStatusProviderImpl(DataStoreStatusProvider): + def __init__(self, store: Optional[FeatureStore], listeners: Listeners): + self.__store = store + self.__listeners = listeners + + self.__lock = ReadWriteLock() + self.__status = DataStoreStatus(True, False) + + def update_status(self, status: DataStoreStatus): + """ + update_status is called from the data store to push a status update. + """ + self.__lock.lock() + modified = False + + if self.__status != status: + self.__status = status + modified = True + + self.__lock.unlock() + + if modified: + self.__listeners.notify(status) + + @property + def status(self) -> DataStoreStatus: + self.__lock.rlock() + status = copy(self.__status) + self.__lock.runlock() + + return status + + def is_monitoring_enabled(self) -> bool: + if self.__store is None: + return False + if hasattr(self.__store, "is_monitoring_enabled") is False: + return False + + return self.__store.is_monitoring_enabled() # type: ignore + + def add_listener(self, listener: Callable[[DataStoreStatus], None]): + self.__listeners.add(listener) + + def remove_listener(self, listener: Callable[[DataStoreStatus], None]): + self.__listeners.remove(listener) + + +class FeatureStoreClientWrapper(FeatureStore): + """Provides additional behavior that the client requires before or after feature store operations. + Currently this just means sorting the data set for init() and dealing with data store status listeners. + """ + + def __init__(self, store: FeatureStore, store_update_sink: DataStoreStatusProviderImpl): + self.store = store + self.__store_update_sink = store_update_sink + self.__monitoring_enabled = self.is_monitoring_enabled() + + # Covers the following variables + self.__lock = ReadWriteLock() + self.__last_available = True + self.__poller: Optional[RepeatingTask] = None + + def init(self, all_data: Mapping[VersionedDataKind, Mapping[str, Dict[Any, Any]]]): + return self.__wrapper(lambda: self.store.init(_FeatureStoreDataSetSorter.sort_all_collections(all_data))) + + def get(self, kind, key, callback): + return self.__wrapper(lambda: self.store.get(kind, key, callback)) + + def all(self, kind, callback): + return self.__wrapper(lambda: self.store.all(kind, callback)) + + def delete(self, kind, key, version): + return self.__wrapper(lambda: self.store.delete(kind, key, version)) + + def upsert(self, kind, item): + return self.__wrapper(lambda: self.store.upsert(kind, item)) + + @property + def initialized(self) -> bool: + return self.store.initialized + + def __wrapper(self, fn: Callable): + try: + return fn() + except BaseException: + if self.__monitoring_enabled: + self.__update_availability(False) + raise + + def __update_availability(self, available: bool): + try: + self.__lock.lock() + if available == self.__last_available: + return + self.__last_available = available + finally: + self.__lock.unlock() + + if available: + log.warning("Persistent store is available again") + + status = DataStoreStatus(available, True) + self.__store_update_sink.update_status(status) + + if available: + try: + self.__lock.lock() + if self.__poller is not None: + self.__poller.stop() + self.__poller = None + finally: + self.__lock.unlock() + + return + + log.warning("Detected persistent store unavailability; updates will be cached until it recovers") + task = RepeatingTask("ldclient.check-availability", 0.5, 0, self.__check_availability) + + self.__lock.lock() + self.__poller = task + self.__poller.start() + self.__lock.unlock() + + def __check_availability(self): + try: + if self.store.is_available(): + self.__update_availability(True) + except BaseException as e: + log.error("Unexpected error from data store status function: %s", e) + + def is_monitoring_enabled(self) -> bool: + """ + This methods determines whether the wrapped store can support enabling monitoring. + + The wrapped store must provide a monitoring_enabled method, which must + be true. But this alone is not sufficient. + + Because this class wraps all interactions with a provided store, it can + technically "monitor" any store. However, monitoring also requires that + we notify listeners when the store is available again. + + We determine this by checking the store's `available?` method, so this + is also a requirement for monitoring support. + + These extra checks won't be necessary once `available` becomes a part + of the core interface requirements and this class no longer wraps every + feature store. + """ + + if not hasattr(self.store, 'is_monitoring_enabled'): + return False + + if not hasattr(self.store, 'is_available'): + return False + + monitoring_enabled = getattr(self.store, 'is_monitoring_enabled') + if not callable(monitoring_enabled): + return False + + return monitoring_enabled() + + +class FDv2(DataSystem): + """ + FDv2 is an implementation of the DataSystem interface that uses the Flag Delivery V2 protocol + for obtaining and keeping data up-to-date. Additionally, it operates with an optional persistent + store in read-only or read/write mode. + """ + + def __init__( + self, + config: Config, + data_system_config: DataSystemConfig, + ): + """ + Initialize a new FDv2 data system. + + :param config: Configuration for initializers and synchronizers + :param persistent_store: Optional persistent store for data persistence + :param store_writable: Whether the persistent store should be written to + :param disabled: Whether the data system is disabled (offline mode) + """ + self._config = config + self._data_system_config = data_system_config + self._primary_synchronizer_builder: Optional[Builder[Synchronizer]] = data_system_config.primary_synchronizer + self._secondary_synchronizer_builder = data_system_config.secondary_synchronizer + self._fdv1_fallback_synchronizer_builder = data_system_config.fdv1_fallback_synchronizer + self._disabled = self._config.offline + + # Diagnostic accumulator provided by client for streaming metrics + self._diagnostic_accumulator: Optional[DiagnosticAccumulator] = None + + # Set up event listeners + self._flag_change_listeners = Listeners() + self._change_set_listeners = Listeners() + self._data_store_listeners = Listeners() + + self._data_store_listeners.add(self._persistent_store_outage_recovery) + + # Create the store + self._store = Store(self._flag_change_listeners, self._change_set_listeners) + + # Status providers + self._data_source_status_provider = DataSourceStatusProviderImpl(Listeners()) + self._data_store_status_provider = DataStoreStatusProviderImpl(None, self._data_store_listeners) + + # Configure persistent store if provided + if self._data_system_config.data_store is not None: + self._data_store_status_provider = DataStoreStatusProviderImpl(self._data_system_config.data_store, self._data_store_listeners) + writable = self._data_system_config.data_store_mode == DataStoreMode.READ_WRITE + wrapper = FeatureStoreClientWrapper(self._data_system_config.data_store, self._data_store_status_provider) + self._store.with_persistence( + wrapper, writable, self._data_store_status_provider + ) + + # Flag tracker (evaluation function set later by client) + self._flag_tracker = FlagTrackerImpl( + self._flag_change_listeners, + lambda key, context: None # Placeholder, replaced by client + ) + + # Threading + self._stop_event = Event() + self._lock = ReadWriteLock() + self._active_synchronizer: Optional[Synchronizer] = None + self._threads: List[Thread] = [] + + # Track configuration + self._configured_with_data_sources = ( + (data_system_config.initializers is not None and len(data_system_config.initializers) > 0) + or data_system_config.primary_synchronizer is not None + ) + + def start(self, set_on_ready: Event): + """ + Start the FDv2 data system. + + :param set_on_ready: Event to set when the system is ready or has failed + """ + if self._disabled: + log.warning("Data system is disabled, SDK will return application-defined default values") + set_on_ready.set() + return + + self._stop_event.clear() + + # Start the main coordination thread + main_thread = Thread( + target=self._run_main_loop, + args=(set_on_ready,), + name="FDv2-main", + daemon=True + ) + main_thread.start() + self._threads.append(main_thread) + + def stop(self): + """Stop the FDv2 data system and all associated threads.""" + self._stop_event.set() + + self._lock.lock() + if self._active_synchronizer is not None: + try: + self._active_synchronizer.stop() + except Exception as e: + log.error("Error stopping active data source: %s", e) + self._lock.unlock() + + # Wait for all threads to complete + for thread in self._threads: + if thread.is_alive(): + thread.join(timeout=5.0) # 5 second timeout + if thread.is_alive(): + log.warning("Thread %s did not terminate in time", thread.name) + + # Close the store + self._store.close() + + def set_diagnostic_accumulator(self, diagnostic_accumulator: DiagnosticAccumulator): + """ + Sets the diagnostic accumulator for streaming initialization metrics. + This should be called before start() to ensure metrics are collected. + """ + self._diagnostic_accumulator = diagnostic_accumulator + + def _run_main_loop(self, set_on_ready: Event): + """Main coordination loop that manages initializers and synchronizers.""" + try: + self._data_source_status_provider.update_status( + DataSourceState.INITIALIZING, None + ) + + # Run initializers first + self._run_initializers(set_on_ready) + + # Run synchronizers + self._run_synchronizers(set_on_ready) + + except Exception as e: + log.error("Error in FDv2 main loop: %s", e) + # Ensure ready event is set even on error + if not set_on_ready.is_set(): + set_on_ready.set() + + def _run_initializers(self, set_on_ready: Event): + """Run initializers to get initial data.""" + if self._data_system_config.initializers is None: + return + + for initializer_builder in self._data_system_config.initializers: + if self._stop_event.is_set(): + return + + try: + initializer = initializer_builder(self._config) + log.info("Attempting to initialize via %s", initializer.name) + + basis_result = initializer.fetch(self._store) + + if isinstance(basis_result, _Fail): + log.warning("Initializer %s failed: %s", initializer.name, basis_result.error) + continue + + basis = basis_result.value + log.info("Initialized via %s", initializer.name) + + # Apply the basis to the store + self._store.apply(basis.change_set, basis.persist) + + # Set ready event + if not set_on_ready.is_set(): + set_on_ready.set() + except Exception as e: + log.error("Initializer failed with exception: %s", e) + + def _run_synchronizers(self, set_on_ready: Event): + """Run synchronizers to keep data up-to-date.""" + # If no primary synchronizer configured, just set ready and return + if self._data_system_config.primary_synchronizer is None: + if not set_on_ready.is_set(): + set_on_ready.set() + return + + def synchronizer_loop(self: 'FDv2'): + try: + # Always ensure ready event is set when we exit + while not self._stop_event.is_set() and self._primary_synchronizer_builder is not None: + # Try primary synchronizer + try: + self._lock.lock() + primary_sync = self._primary_synchronizer_builder(self._config) + if isinstance(primary_sync, DiagnosticSource) and self._diagnostic_accumulator is not None: + primary_sync.set_diagnostic_accumulator(self._diagnostic_accumulator) + self._active_synchronizer = primary_sync + self._lock.unlock() + + log.info("Primary synchronizer %s is starting", primary_sync.name) + + remove_sync, fallback_v1 = self._consume_synchronizer_results( + primary_sync, set_on_ready, self._fallback_condition + ) + + if remove_sync: + self._primary_synchronizer_builder = self._secondary_synchronizer_builder + self._secondary_synchronizer_builder = None + + if fallback_v1: + self._primary_synchronizer_builder = self._fdv1_fallback_synchronizer_builder + + if self._primary_synchronizer_builder is None: + log.warning("No more synchronizers available") + self._data_source_status_provider.update_status( + DataSourceState.OFF, + self._data_source_status_provider.status.error + ) + break + else: + log.info("Fallback condition met") + + if self._stop_event.is_set(): + break + + if self._secondary_synchronizer_builder is None: + continue + + self._lock.lock() + secondary_sync = self._secondary_synchronizer_builder(self._config) + if isinstance(secondary_sync, DiagnosticSource) and self._diagnostic_accumulator is not None: + secondary_sync.set_diagnostic_accumulator(self._diagnostic_accumulator) + log.info("Secondary synchronizer %s is starting", secondary_sync.name) + self._active_synchronizer = secondary_sync + self._lock.unlock() + + remove_sync, fallback_v1 = self._consume_synchronizer_results( + secondary_sync, set_on_ready, self._recovery_condition + ) + + if remove_sync: + self._secondary_synchronizer_builder = None + if fallback_v1: + self._primary_synchronizer_builder = self._fdv1_fallback_synchronizer_builder + + if self._primary_synchronizer_builder is None: + log.warning("No more synchronizers available") + self._data_source_status_provider.update_status( + DataSourceState.OFF, + self._data_source_status_provider.status.error + ) + break + + log.info("Recovery condition met, returning to primary synchronizer") + except Exception as e: + log.error("Failed to build primary synchronizer: %s", e) + break + + except Exception as e: + log.error("Error in synchronizer loop: %s", e) + finally: + # Ensure we always set the ready event when exiting + set_on_ready.set() + self._lock.lock() + if self._active_synchronizer is not None: + self._active_synchronizer.stop() + self._active_synchronizer = None + self._lock.unlock() + + sync_thread = Thread( + target=synchronizer_loop, + name="FDv2-synchronizers", + args=(self,), + daemon=True + ) + sync_thread.start() + self._threads.append(sync_thread) + + def _consume_synchronizer_results( + self, + synchronizer: Synchronizer, + set_on_ready: Event, + condition_func: Callable[[DataSourceStatus], bool] + ) -> tuple[bool, bool]: + """ + Consume results from a synchronizer until a condition is met or it fails. + + :return: Tuple of (should_remove_sync, fallback_to_fdv1) + """ + action_queue: Queue = Queue() + timer = RepeatingTask( + label="FDv2-sync-cond-timer", + interval=10, + initial_delay=10, + callable=lambda: action_queue.put("check") + ) + + def reader(self: 'FDv2'): + try: + for update in synchronizer.sync(self._store): + action_queue.put(update) + finally: + action_queue.put("quit") + + sync_reader = Thread( + target=reader, + name="FDv2-sync-reader", + args=(self,), + daemon=True + ) + + try: + timer.start() + sync_reader.start() + + while True: + update = action_queue.get(True) + if isinstance(update, str): + if update == "quit": + break + + if update == "check": + # Check condition periodically + current_status = self._data_source_status_provider.status + if condition_func(current_status): + return False, False + continue + + log.info("Synchronizer %s update: %s", synchronizer.name, update.state) + if self._stop_event.is_set(): + return False, False + + # Handle the update + if update.change_set is not None: + self._store.apply(update.change_set, True) + + # Set ready event on first valid update + if update.state == DataSourceState.VALID and not set_on_ready.is_set(): + set_on_ready.set() + + # Update status + self._data_source_status_provider.update_status(update.state, update.error) + + # Check if we should revert to FDv1 immediately + if update.revert_to_fdv1: + return True, True + + # Check for OFF state indicating permanent failure + if update.state == DataSourceState.OFF: + return True, False + except Exception as e: + log.error("Error consuming synchronizer results: %s", e) + return True, False + finally: + synchronizer.stop() + timer.stop() + + sync_reader.join(0.5) + + return True, False + + def _fallback_condition(self, status: DataSourceStatus) -> bool: + """ + Determine if we should fallback to secondary synchronizer. + + :param status: Current data source status + :return: True if fallback condition is met + """ + interrupted_at_runtime = ( + status.state == DataSourceState.INTERRUPTED + and time.time() - status.since > 60 # 1 minute + ) + cannot_initialize = ( + status.state == DataSourceState.INITIALIZING + and time.time() - status.since > 10 # 10 seconds + ) + + return interrupted_at_runtime or cannot_initialize + + def _recovery_condition(self, status: DataSourceStatus) -> bool: + """ + Determine if we should try to recover to primary synchronizer. + + :param status: Current data source status + :return: True if recovery condition is met + """ + interrupted_at_runtime = ( + status.state == DataSourceState.INTERRUPTED + and time.time() - status.since > 60 # 1 minute + ) + healthy_for_too_long = ( + status.state == DataSourceState.VALID + and time.time() - status.since > 300 # 5 minutes + ) + cannot_initialize = ( + status.state == DataSourceState.INITIALIZING + and time.time() - status.since > 10 # 10 seconds + ) + + return interrupted_at_runtime or healthy_for_too_long or cannot_initialize + + def _persistent_store_outage_recovery(self, data_store_status: DataStoreStatus): + """ + Monitor the data store status. If the store comes online and + potentially has stale data, we should write our known state to it. + """ + if not data_store_status.available: + return + + if not data_store_status.stale: + return + + err = self._store.commit() + if err is not None: + log.error("Failed to reinitialize data store", exc_info=err) + + @property + def store(self) -> ReadOnlyStore: + """Get the underlying store for flag evaluation.""" + return self._store.get_active_store() + + def set_flag_value_eval_fn(self, eval_fn): + """ + Set the flag value evaluation function for the flag tracker. + + :param eval_fn: Function with signature (key: str, context: Context) -> Any + """ + self._flag_tracker = FlagTrackerImpl(self._flag_change_listeners, eval_fn) + + @property + def data_source_status_provider(self) -> DataSourceStatusProvider: + """Get the data source status provider.""" + return self._data_source_status_provider + + @property + def data_store_status_provider(self) -> DataStoreStatusProvider: + """Get the data store status provider.""" + return self._data_store_status_provider + + @property + def flag_tracker(self) -> FlagTracker: + """Get the flag tracker for monitoring flag changes.""" + return self._flag_tracker + + @property + def data_availability(self) -> DataAvailability: + """Get the current data availability level.""" + if self._store.selector().is_defined(): + return DataAvailability.REFRESHED + + if not self._configured_with_data_sources or self._store.is_initialized(): + return DataAvailability.CACHED + + return DataAvailability.DEFAULTS + + @property + def target_availability(self) -> DataAvailability: + """Get the target data availability level based on configuration.""" + if self._configured_with_data_sources: + return DataAvailability.REFRESHED + + return DataAvailability.CACHED diff --git a/ldclient/impl/datasystem/protocolv2.py b/ldclient/impl/datasystem/protocolv2.py index 50cc0862..55736430 100644 --- a/ldclient/impl/datasystem/protocolv2.py +++ b/ldclient/impl/datasystem/protocolv2.py @@ -3,182 +3,9 @@ LaunchDarkly data system version 2 (FDv2). """ -from abc import abstractmethod from dataclasses import dataclass -from enum import Enum -from typing import Any, List, Optional, Protocol -from ldclient.impl.util import Result - - -class EventName(str, Enum): - """ - EventName represents the name of an event that can be sent by the server for FDv2. - """ - - PUT_OBJECT = "put-object" - """ - Specifies that an object should be added to the data set with upsert semantics. - """ - - DELETE_OBJECT = "delete-object" - """ - Specifies that an object should be removed from the data set. - """ - - SERVER_INTENT = "server-intent" - """ - Specifies the server's intent. - """ - - PAYLOAD_TRANSFERRED = "payload-transferred" - """ - Specifies that that all data required to bring the existing data set to - a new version has been transferred. - """ - - HEARTBEAT = "heart-beat" - """ - Keeps the connection alive. - """ - - GOODBYE = "goodbye" - """ - Specifies that the server is about to close the connection. - """ - - ERROR = "error" - """ - Specifies that an error occurred while serving the connection. - """ - - -class IntentCode(str, Enum): - """ - IntentCode represents the various intents that can be sent by the server. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - TRANSFER_FULL = "xfer-full" - """ - The server intends to send a full data set. - """ - TRANSFER_CHANGES = "xfer-changes" - """ - The server intends to send only the necessary changes to bring an existing - data set up-to-date. - """ - - TRANSFER_NONE = "none" - """ - The server intends to send no data (payload is up to date). - """ - - -@dataclass(frozen=True) -class Payload: - """ - Payload represents a payload delivered in a streaming response. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - id: str - target: int - code: IntentCode - reason: str - - def to_dict(self) -> dict: - """ - Serializes the Payload to a JSON-compatible dictionary. - """ - return { - "id": self.id, - "target": self.target, - "intentCode": self.code.value, - "reason": self.reason, - } - - @staticmethod - def from_dict(data: dict) -> "Payload": - """ - Create a Payload from a dictionary representation. - """ - intent_code = data.get("intentCode") - - if intent_code is None or not isinstance(intent_code, str): - raise ValueError( - "Invalid data for Payload: 'intentCode' key is missing or not a string" - ) - - return Payload( - id=data.get("id", ""), - target=data.get("target", 0), - code=IntentCode(intent_code), - reason=data.get("reason", ""), - ) - - -@dataclass(frozen=True) -class ServerIntent: - """ - ServerIntent represents the type of change associated with the payload - (e.g., transfer full, transfer changes, etc.) - """ - - payload: Payload - - def to_dict(self) -> dict: - """ - Serializes the ServerIntent to a JSON-compatible dictionary. - """ - return { - "payloads": [self.payload.to_dict()], - } - - @staticmethod - def from_dict(data: dict) -> "ServerIntent": - """ - Create a ServerIntent from a dictionary representation. - """ - if "payloads" not in data or not isinstance(data["payloads"], list): - raise ValueError( - "Invalid data for ServerIntent: 'payloads' key is missing or not a list" - ) - if len(data["payloads"]) != 1: - raise ValueError( - "Invalid data for ServerIntent: expected exactly one payload" - ) - - payload = data["payloads"][0] - if not isinstance(payload, dict): - raise ValueError("Invalid payload in ServerIntent: expected a dictionary") - - return ServerIntent(payload=Payload.from_dict(payload)) - - -class ObjectKind(str, Enum): - """ - ObjectKind represents the kind of object. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - FLAG = "flag" - SEGMENT = "segment" +from ldclient.interfaces import EventName, ObjectKind @dataclass(frozen=True) @@ -357,285 +184,3 @@ def from_dict(data: dict) -> "Error": raise ValueError("Missing required fields in Error JSON.") return Error(payload_id=payload_id, reason=reason) - - -@dataclass(frozen=True) -class Selector: - """ - Selector represents a particular snapshot of data. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - state: str = "" - version: int = 0 - - @staticmethod - def no_selector() -> "Selector": - """ - Returns an empty Selector. - """ - return Selector() - - def is_defined(self) -> bool: - """ - Returns True if the Selector has a value. - """ - return self != Selector.no_selector() - - def name(self) -> str: - """ - Event method. - """ - return EventName.PAYLOAD_TRANSFERRED - - @staticmethod - def new_selector(state: str, version: int) -> "Selector": - """ - Creates a new Selector from a state string and version. - """ - return Selector(state=state, version=version) - - def to_dict(self) -> dict: - """ - Serializes the Selector to a JSON-compatible dictionary. - """ - return {"state": self.state, "version": self.version} - - @staticmethod - def from_dict(data: dict) -> "Selector": - """ - Deserializes a Selector from a JSON-compatible dictionary. - """ - state = data.get("state") - version = data.get("version") - - if state is None or version is None: - raise ValueError("Missing required fields in Selector JSON.") - - return Selector(state=state, version=version) - - -class ChangeType(Enum): - """ - ChangeType specifies if an object is being upserted or deleted. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - PUT = "put" - """ - Represents an object being upserted. - """ - - DELETE = "delete" - """ - Represents an object being deleted. - """ - - -@dataclass(frozen=True) -class Change: - """ - Change represents a change to a piece of data, such as an update or deletion. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - action: ChangeType - kind: ObjectKind - key: str - version: int - object: Any = ( - None # TODO(fdv2): At some point, we should define a better type for this. - ) - - -@dataclass(frozen=True) -class ChangeSet: - """ - ChangeSet represents a list of changes to be applied. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - intent_code: IntentCode - changes: List[Change] - selector: Optional[Selector] - - -@dataclass(frozen=True) -class Basis: - """ - Basis represents the initial payload of data that a data source can - provide. Initializers provide this via fetch, whereas Synchronizers provide - it asynchronously. - """ - - change_set: ChangeSet - persist: bool - environment_id: Optional[str] = None - - -class Synchronizer(Protocol): - """ - Represents a component capable of obtaining a Basis and subsequent delta - updates asynchronously. - """ - - @abstractmethod - def name(self) -> str: - """Returns the name of the initializer.""" - raise NotImplementedError - - # TODO(fdv2): Need sync method - - def close(self): - """ - Close the synchronizer, releasing any resources it holds. - """ - - -class Initializer(Protocol): - """ - Represents a component capable of obtaining a Basis via a synchronous call. - """ - - @abstractmethod - def name(self) -> str: - """Returns the name of the initializer.""" - raise NotImplementedError - - @abstractmethod - def fetch(self) -> Result: - """ - Fetch returns a Basis, or an error if the Basis could not be retrieved. - """ - raise NotImplementedError - - -class ChangeSetBuilder: - """ - ChangeSetBuilder is a helper for constructing a ChangeSet. - - This type is not stable, and not subject to any backwards - compatibility guarantees or semantic versioning. It is not suitable for production usage. - - Do not use it. - You have been warned. - """ - - def __init__(self): - """ - Initializes a new ChangeSetBuilder. - """ - self.intent = None - self.changes = [] - - @staticmethod - def no_changes() -> "ChangeSet": - """ - Represents an intent that the current data is up-to-date and doesn't - require changes. - """ - return ChangeSet( - intent_code=IntentCode.TRANSFER_NONE, selector=None, changes=[] - ) - - @staticmethod - def empty(selector) -> "ChangeSet": - """ - Returns an empty ChangeSet, which is useful for initializing a client - without data or for clearing out all existing data. - """ - return ChangeSet( - intent_code=IntentCode.TRANSFER_FULL, selector=selector, changes=[] - ) - - def start(self, intent: IntentCode): - """ - Begins a new change set with a given intent. - """ - self.intent = intent - self.changes = [] - - def expect_changes(self): - """ - Ensures that the current ChangeSetBuilder is prepared to handle changes. - - If a data source's initial connection reflects an updated status, we - need to keep the provided server intent. This allows subsequent changes - to come down the line without an explicit server intent. - - However, to maintain logical consistency, we need to ensure that the intent - is set to IntentTransferChanges. - """ - if self.intent is None: - raise ValueError("changeset: cannot expect changes without a server-intent") - - if self.intent != IntentCode.TRANSFER_NONE: - return - - self.intent = IntentCode.TRANSFER_CHANGES - - def reset(self): - """ - Clears any existing changes while preserving the current intent. - """ - self.changes = [] - - def finish(self, selector) -> ChangeSet: - """ - Identifies a changeset with a selector and returns the completed - changeset. Clears any existing changes while preserving the current - intent, so the builder can be reused. - """ - if self.intent is None: - raise ValueError("changeset: cannot complete without a server-intent") - - changeset = ChangeSet( - intent_code=self.intent, selector=selector, changes=self.changes - ) - self.changes = [] - - # Once a full transfer has been processed, all future changes should be - # assumed to be changes. Flag delivery can override this behavior by - # sending a new server intent to any connected stream. - if self.intent == IntentCode.TRANSFER_FULL: - self.intent = IntentCode.TRANSFER_CHANGES - - return changeset - - def add_put(self, kind, key, version, obj): - """ - Adds a new object to the changeset. - """ - self.changes.append( - Change( - action=ChangeType.PUT, kind=kind, key=key, version=version, object=obj - ) - ) - - def add_delete(self, kind, key, version): - """ - Adds a deletion to the changeset. - """ - self.changes.append( - Change(action=ChangeType.DELETE, kind=kind, key=key, version=version) - ) diff --git a/ldclient/impl/datasystem/store.py b/ldclient/impl/datasystem/store.py new file mode 100644 index 00000000..0d731e03 --- /dev/null +++ b/ldclient/impl/datasystem/store.py @@ -0,0 +1,483 @@ +""" +Store implementation for FDv2 data system. + +This module provides a dual-mode persistent/in-memory store that serves requests for data +from the evaluation algorithm. It manages both in-memory and persistent storage, handling +ChangeSet applications and flag change notifications. +""" + +import threading +from collections import defaultdict +from typing import Any, Callable, Dict, List, Optional, Set + +from ldclient.impl.dependency_tracker import DependencyTracker, KindAndKey +from ldclient.impl.listeners import Listeners +from ldclient.impl.model.entity import ModelEntity +from ldclient.impl.rwlock import ReadWriteLock +from ldclient.impl.util import log +from ldclient.interfaces import ( + Change, + ChangeSet, + ChangeType, + DataStoreStatusProvider, + FeatureStore, + FlagChange, + IntentCode, + ObjectKind, + ReadOnlyStore, + Selector +) +from ldclient.versioned_data_kind import FEATURES, SEGMENTS, VersionedDataKind + +Collections = Dict[VersionedDataKind, Dict[str, dict]] + + +class InMemoryFeatureStore(ReadOnlyStore): + """ + The default feature store implementation, which holds all data in a + thread-safe data structure in memory. + """ + + def __init__(self): + """Constructs an instance of InMemoryFeatureStore.""" + self._lock = ReadWriteLock() + self._initialized = False + self._items = defaultdict(dict) + + def get( + self, + kind: VersionedDataKind, + key: str, + callback: Callable[[Any], Any] = lambda x: x, + ) -> Any: + try: + self._lock.rlock() + items_of_kind = self._items[kind] + item = items_of_kind.get(key) + if item is None: + log.debug( + "Attempted to get missing key %s in '%s', returning None", + key, + kind.namespace, + ) + return callback(None) + if "deleted" in item and item["deleted"]: + log.debug( + "Attempted to get deleted key %s in '%s', returning None", + key, + kind.namespace, + ) + return callback(None) + return callback(item) + finally: + self._lock.runlock() + + def all(self, kind: VersionedDataKind, callback: Callable[[Any], Any] = lambda x: x) -> Any: + try: + self._lock.rlock() + items_of_kind = self._items[kind] + return callback( + dict( + (k, i) + for k, i in items_of_kind.items() + if ("deleted" not in i) or not i["deleted"] + ) + ) + finally: + self._lock.runlock() + + def set_basis(self, collections: Collections) -> bool: + """ + Initializes the store with a full set of data, replacing any existing data. + """ + all_decoded = self.__decode_collection(collections) + if all_decoded is None: + return False + + try: + self._lock.lock() + self._items.clear() + self._items.update(all_decoded) + self._initialized = True + except Exception as e: + log.error("Failed applying set_basis", exc_info=e) + return False + finally: + self._lock.unlock() + + return True + + def apply_delta(self, collections: Collections) -> bool: + """ + Applies a delta update to the store. + """ + all_decoded = self.__decode_collection(collections) + if all_decoded is None: + return False + + try: + self._lock.lock() + for kind, kind_data in all_decoded.items(): + items_of_kind = self._items[kind] + kind_data = all_decoded[kind] + for key, item in kind_data.items(): + items_of_kind[key] = item + log.debug( + "Updated %s in '%s' to version %d", key, kind.namespace, item["version"] + ) + except Exception as e: + log.error("Failed applying apply_delta", exc_info=e) + return False + finally: + self._lock.unlock() + + return True + + def __decode_collection(self, collections: Collections) -> Optional[Dict[VersionedDataKind, Dict[str, Any]]]: + try: + all_decoded = {} + for kind in collections: + collection = collections[kind] + items_decoded = {} + for key in collection: + items_decoded[key] = kind.decode(collection[key]) + all_decoded[kind] = items_decoded + + return all_decoded + except Exception as e: + log.error("Failed decoding collection.", exc_info=e) + return None + + @property + def initialized(self) -> bool: + """ + Indicates whether the store has been initialized with data. + """ + try: + self._lock.rlock() + return self._initialized + finally: + self._lock.runlock() + + +class Store: + """ + Store is a dual-mode persistent/in-memory store that serves requests for + data from the evaluation algorithm. + + At any given moment one of two stores is active: in-memory, or persistent. + Once the in-memory store has data (either from initializers or a + synchronizer), the persistent store is no longer read from. From that point + forward, calls to get data will serve from the memory store. + """ + + def __init__( + self, + flag_change_listeners: Listeners, + change_set_listeners: Listeners, + ): + """ + Initialize a new Store. + + Args: + flag_change_listeners: Listeners for flag change events + change_set_listeners: Listeners for changeset events + """ + self._persistent_store: Optional[FeatureStore] = None + self._persistent_store_status_provider: Optional[DataStoreStatusProvider] = None + self._persistent_store_writable = False + + # Source of truth for flag evaluations once initialized + self._memory_store = InMemoryFeatureStore() + + # Used to track dependencies between items in the store + self._dependency_tracker = DependencyTracker() + + # Listeners for events + self._flag_change_listeners = flag_change_listeners + self._change_set_listeners = change_set_listeners + + # True if the data in the memory store may be persisted to the persistent store + self._persist = False + + # Points to the active store. Swapped upon initialization. + self._active_store: ReadOnlyStore = self._memory_store + + # Identifies the current data + self._selector = Selector.no_selector() + + # Thread synchronization + self._lock = threading.RLock() + + def with_persistence( + self, + persistent_store: FeatureStore, + writable: bool, + status_provider: Optional[DataStoreStatusProvider] = None, + ) -> "Store": + """ + Configure the store with a persistent store for read-only or read-write access. + + Args: + persistent_store: The persistent store implementation + writable: Whether the persistent store should be written to + status_provider: Optional status provider for the persistent store + + Returns: + Self for method chaining + """ + with self._lock: + self._persistent_store = persistent_store + self._persistent_store_writable = writable + self._persistent_store_status_provider = status_provider + + # Initially use persistent store as active until memory store has data + self._active_store = persistent_store + + return self + + def selector(self) -> Selector: + """Returns the current selector.""" + with self._lock: + return self._selector + + def close(self) -> Optional[Exception]: + """Close the store and any persistent store if configured.""" + with self._lock: + if self._persistent_store is not None: + try: + # Most FeatureStore implementations don't have close methods + # but we'll try to call it if it exists + if hasattr(self._persistent_store, "close"): + self._persistent_store.close() + except Exception as e: + return e + return None + + def apply(self, change_set: ChangeSet, persist: bool) -> None: + """ + Apply a changeset to the store. + + Args: + change_set: The changeset to apply + persist: Whether the changes should be persisted to the persistent store + """ + collections = self._changes_to_store_data(change_set.changes) + + with self._lock: + try: + if change_set.intent_code == IntentCode.TRANSFER_FULL: + self._set_basis(collections, change_set.selector, persist) + elif change_set.intent_code == IntentCode.TRANSFER_CHANGES: + self._apply_delta(collections, change_set.selector, persist) + elif change_set.intent_code == IntentCode.TRANSFER_NONE: + # No-op, no changes to apply + return + + # Notify changeset listeners + self._change_set_listeners.notify(change_set) + + except Exception as e: + # Log error but don't re-raise - matches Go behavior + log.error("Store: couldn't apply changeset: %s", str(e)) + + def _set_basis( + self, collections: Collections, selector: Optional[Selector], persist: bool + ) -> None: + """ + Set the basis of the store. Any existing data is discarded. + + Args: + change_set: The changeset containing the new basis data + persist: Whether to persist the data to the persistent store + """ + # Take snapshot for change detection if we have flag listeners + old_data: Optional[Collections] = None + if self._flag_change_listeners.has_listeners(): + old_data = {} + for kind in [FEATURES, SEGMENTS]: + old_data[kind] = self._memory_store.all(kind, lambda x: x) + + ok = self._memory_store.set_basis(collections) + if ok is False: + return + + # Update dependency tracker + self._reset_dependency_tracker(collections) + + # Update state + self._persist = persist + self._selector = selector if selector is not None else Selector.no_selector() + + # Switch to memory store as active + self._active_store = self._memory_store + + # Persist to persistent store if configured and writable + if self._should_persist(): + self._persistent_store.init(collections) # type: ignore + + # Send change events if we had listeners + if old_data is not None: + affected_items = self._compute_changed_items_for_full_data_set( + old_data, collections + ) + self._send_change_events(affected_items) + + def _apply_delta( + self, collections: Collections, selector: Optional[Selector], persist: bool + ) -> None: + """ + Apply a delta update to the store. + + Args: + change_set: The changeset containing the delta changes + persist: Whether to persist the changes to the persistent store + """ + ok = self._memory_store.apply_delta(collections) + if ok is False: + return + + has_listeners = self._flag_change_listeners.has_listeners() + affected_items: Set[KindAndKey] = set() + + for kind in collections: + collection = collections[kind] + for key in collection: + item = collection[key] + self._dependency_tracker.update_dependencies_from( + kind, key, item + ) + if has_listeners: + self._dependency_tracker.add_affected_items( + affected_items, KindAndKey(kind=kind, key=key) + ) + + # Update state + self._persist = persist + self._selector = selector if selector is not None else Selector.no_selector() + + if self._should_persist(): + for kind in collections: + kind_data: Dict[str, dict] = collections[kind] + for i in kind_data: + item = kind_data[i] + self._persistent_store.upsert(kind, item) # type: ignore + + # Send change events + if affected_items: + self._send_change_events(affected_items) + + def _should_persist(self) -> bool: + """Returns whether data should be persisted to the persistent store.""" + return ( + self._persist + and self._persistent_store is not None + and self._persistent_store_writable + ) + + def _changes_to_store_data(self, changes: List[Change]) -> Collections: + """ + Convert a list of Changes to the pre-existing format used by FeatureStore. + + Args: + changes: List of changes to convert + + Returns: + Mapping suitable for FeatureStore operations. + """ + all_data: Collections = { + FEATURES: {}, + SEGMENTS: {}, + } + + for change in changes: + kind = FEATURES if change.kind == ObjectKind.FLAG else SEGMENTS + if change.action == ChangeType.PUT and change.object is not None: + all_data[kind][change.key] = change.object + if change.action == ChangeType.DELETE: + all_data[kind][change.key] = {'key': change.key, 'deleted': True, 'version': change.version} + + return all_data + + def _reset_dependency_tracker(self, all_data: Collections) -> None: + """Reset dependency tracker with new full data set.""" + self._dependency_tracker.reset() + for kind, items in all_data.items(): + for key, item in items.items(): + self._dependency_tracker.update_dependencies_from(kind, key, item) + + def _send_change_events(self, affected_items: Set[KindAndKey]) -> None: + """Send flag change events for affected items.""" + for item in affected_items: + if item.kind == FEATURES: + self._flag_change_listeners.notify(FlagChange(item.key)) + + def _compute_changed_items_for_full_data_set( + self, + old_data: Collections, + new_data: Collections, + ) -> Set[KindAndKey]: + """Compute which items changed between old and new data sets.""" + affected_items: Set[KindAndKey] = set() + + for kind in [FEATURES, SEGMENTS]: + old_items = old_data.get(kind, {}) + new_items = new_data.get(kind, {}) + + # Get all keys from both old and new data + all_keys = set(old_items.keys()) | set(new_items.keys()) + + for key in all_keys: + old_item = old_items.get(key) + new_item = new_items.get(key) + + # If either is missing or versions differ, it's a change + if old_item is None or new_item is None: + self._dependency_tracker.add_affected_items( + affected_items, KindAndKey(kind=kind, key=key) + ) + elif old_item.get("version", 0) != new_item.get("version", 0): + self._dependency_tracker.add_affected_items( + affected_items, KindAndKey(kind=kind, key=key) + ) + + return affected_items + + def commit(self) -> Optional[Exception]: + """ + Commit persists the data in the memory store to the persistent store, if configured. + + Returns: + Exception if commit failed, None otherwise + """ + def __mapping_from_kind(kind: VersionedDataKind) -> Callable[[Dict[str, ModelEntity]], Dict[str, Dict[str, Any]]]: + def __mapping(data: Dict[str, ModelEntity]) -> Dict[str, Dict[str, Any]]: + return {k: kind.encode(v) for k, v in data.items()} + + return __mapping + + with self._lock: + if self._should_persist(): + try: + # Get all data from memory store and write to persistent store + all_data = {} + for kind in [FEATURES, SEGMENTS]: + all_data[kind] = self._memory_store.all(kind, __mapping_from_kind(kind)) + self._persistent_store.init(all_data) # type: ignore + except Exception as e: + return e + return None + + def get_active_store(self) -> ReadOnlyStore: + """Get the currently active store for reading data.""" + with self._lock: + return self._active_store + + def is_initialized(self) -> bool: + """Check if the active store is initialized.""" + return self.get_active_store().initialized + + def get_data_store_status_provider(self) -> Optional[DataStoreStatusProvider]: + """Get the data store status provider for the persistent store, if configured.""" + with self._lock: + return self._persistent_store_status_provider diff --git a/ldclient/impl/dependency_tracker.py b/ldclient/impl/dependency_tracker.py index 1f6286b2..23d6b0d5 100644 --- a/ldclient/impl/dependency_tracker.py +++ b/ldclient/impl/dependency_tracker.py @@ -89,7 +89,7 @@ def compute_dependencies_from(from_kind: VersionedDataKind, from_item: Optional[ @param from_item [LaunchDarkly::Impl::Model::FeatureFlag, LaunchDarkly::Impl::Model::Segment] @return [Set] """ - if from_item is None: + if from_item is None or from_item.get('deleted', False): return set() from_item = from_kind.decode(from_item) if isinstance(from_item, dict) else from_item diff --git a/ldclient/impl/integrations/files/file_data_sourcev2.py b/ldclient/impl/integrations/files/file_data_sourcev2.py new file mode 100644 index 00000000..5ea976ed --- /dev/null +++ b/ldclient/impl/integrations/files/file_data_sourcev2.py @@ -0,0 +1,438 @@ +import json +import os +import threading +import traceback +from queue import Empty, Queue +from typing import Generator + +from ldclient.impl.repeating_task import RepeatingTask +from ldclient.impl.util import _Fail, _Success, current_time_millis, log +from ldclient.interfaces import ( + Basis, + BasisResult, + ChangeSetBuilder, + DataSourceErrorInfo, + DataSourceErrorKind, + DataSourceState, + IntentCode, + ObjectKind, + Selector, + SelectorStore, + Update +) + +have_yaml = False +try: + import yaml + + have_yaml = True +except ImportError: + pass + +have_watchdog = False +try: + import watchdog + import watchdog.events + import watchdog.observers + + have_watchdog = True +except ImportError: + pass + + +def _sanitize_json_item(item): + if not ("version" in item): + item["version"] = 1 + + +class _FileDataSourceV2: + """ + Internal implementation of both Initializer and Synchronizer protocols for file-based data. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + + This component reads feature flag and segment data from local files and provides them + via the FDv2 protocol interfaces. Each instance implements both Initializer and Synchronizer + protocols: + - As an Initializer: reads files once and returns initial data + - As a Synchronizer: watches for file changes and yields updates + + The files use the same format as the v1 file data source, supporting flags, flagValues, + and segments in JSON or YAML format. + """ + + def __init__(self, paths, poll_interval=1, force_polling=False): + """ + Initialize the file data source. + + :param paths: list of file paths to load (or a single path string) + :param poll_interval: seconds between polling checks when watching files (default: 1) + :param force_polling: force polling even if watchdog is available (default: False) + """ + self._paths = paths if isinstance(paths, list) else [paths] + self._poll_interval = poll_interval + self._force_polling = force_polling + self._closed = False + self._update_queue = Queue() + self._lock = threading.Lock() + self._auto_updater = None + + @property + def name(self) -> str: + """Return the name of this data source.""" + return "FileDataV2" + + def fetch(self, ss: SelectorStore) -> BasisResult: + """ + Implementation of the Initializer.fetch method. + + Reads all configured files once and returns their contents as a Basis. + + :param ss: SelectorStore (not used, as we don't have selectors for file data) + :return: BasisResult containing the file data or an error + """ + try: + with self._lock: + if self._closed: + return _Fail("FileDataV2 source has been closed") + + # Load all files and build changeset + result = self._load_all_to_changeset() + if isinstance(result, _Fail): + return result + + change_set = result.value + + basis = Basis(change_set=change_set, persist=False, environment_id=None) + + return _Success(basis) + + except Exception as e: + log.error("Error fetching file data: %s" % repr(e)) + traceback.print_exc() + return _Fail(f"Error fetching file data: {str(e)}") + + def sync(self, ss: SelectorStore) -> Generator[Update, None, None]: + """ + Implementation of the Synchronizer.sync method. + + Yields initial data from files, then continues to watch for file changes + and yield updates when files are modified. + + :param ss: SelectorStore (not used, as we don't have selectors for file data) + :return: Generator yielding Update objects + """ + # First yield initial data + initial_result = self.fetch(ss) + if isinstance(initial_result, _Fail): + yield Update( + state=DataSourceState.OFF, + error=DataSourceErrorInfo( + kind=DataSourceErrorKind.INVALID_DATA, + status_code=0, + time=current_time_millis(), + message=initial_result.error, + ), + ) + return + + # Yield the initial successful state + yield Update( + state=DataSourceState.VALID, change_set=initial_result.value.change_set + ) + + # Start watching for file changes + with self._lock: + if not self._closed: + self._auto_updater = self._start_auto_updater() + + # Continue yielding updates as they arrive + while not self._closed: + try: + # Wait for updates with a timeout to allow checking closed status + try: + update = self._update_queue.get(timeout=1.0) + except Empty: + continue + + if update is None: # Sentinel value for shutdown + break + + yield update + + except Exception as e: + log.error("Error in file data synchronizer: %s" % repr(e)) + traceback.print_exc() + yield Update( + state=DataSourceState.OFF, + error=DataSourceErrorInfo( + kind=DataSourceErrorKind.UNKNOWN, + status_code=0, + time=current_time_millis(), + message=f"Error in file data synchronizer: {str(e)}", + ), + ) + break + + def stop(self): + """Stop the data source and clean up resources.""" + with self._lock: + if self._closed: + return + self._closed = True + + auto_updater = self._auto_updater + self._auto_updater = None + + if auto_updater: + auto_updater.stop() + + # Signal shutdown to sync generator + self._update_queue.put(None) + + def _load_all_to_changeset(self): + """ + Load all files and build a changeset. + + :return: _Result containing ChangeSet or error string + """ + flags_dict = {} + segments_dict = {} + + for path in self._paths: + try: + self._load_file(path, flags_dict, segments_dict) + except Exception as e: + log.error('Unable to load flag data from "%s": %s' % (path, repr(e))) + traceback.print_exc() + return _Fail(f'Unable to load flag data from "{path}": {str(e)}') + + # Build a full transfer changeset + builder = ChangeSetBuilder() + builder.start(IntentCode.TRANSFER_FULL) + + # Add all flags to the changeset + for key, flag_data in flags_dict.items(): + builder.add_put( + ObjectKind.FLAG, key, flag_data.get("version", 1), flag_data + ) + + # Add all segments to the changeset + for key, segment_data in segments_dict.items(): + builder.add_put( + ObjectKind.SEGMENT, key, segment_data.get("version", 1), segment_data + ) + + # Use no_selector since we don't have versioning information from files + change_set = builder.finish(Selector.no_selector()) + + return _Success(change_set) + + def _load_file(self, path, flags_dict, segments_dict): + """ + Load a single file and add its contents to the provided dictionaries. + + :param path: path to the file + :param flags_dict: dictionary to add flags to + :param segments_dict: dictionary to add segments to + """ + content = None + with open(path, "r") as f: + content = f.read() + parsed = self._parse_content(content) + + for key, flag in parsed.get("flags", {}).items(): + _sanitize_json_item(flag) + self._add_item(flags_dict, "flags", flag) + + for key, value in parsed.get("flagValues", {}).items(): + self._add_item(flags_dict, "flags", self._make_flag_with_value(key, value)) + + for key, segment in parsed.get("segments", {}).items(): + _sanitize_json_item(segment) + self._add_item(segments_dict, "segments", segment) + + def _parse_content(self, content): + """ + Parse file content as JSON or YAML. + + :param content: file content string + :return: parsed dictionary + """ + if have_yaml: + return yaml.safe_load(content) # pyyaml correctly parses JSON too + return json.loads(content) + + def _add_item(self, items_dict, kind_name, item): + """ + Add an item to a dictionary, checking for duplicates. + + :param items_dict: dictionary to add to + :param kind_name: name of the kind (for error messages) + :param item: item to add + """ + key = item.get("key") + if items_dict.get(key) is None: + items_dict[key] = item + else: + raise Exception( + 'In %s, key "%s" was used more than once' % (kind_name, key) + ) + + def _make_flag_with_value(self, key, value): + """ + Create a simple flag configuration from a key-value pair. + + :param key: flag key + :param value: flag value + :return: flag dictionary + """ + return { + "key": key, + "version": 1, + "on": True, + "fallthrough": {"variation": 0}, + "variations": [value], + } + + def _start_auto_updater(self): + """ + Start watching files for changes. + + :return: auto-updater instance + """ + resolved_paths = [] + for path in self._paths: + try: + resolved_paths.append(os.path.realpath(path)) + except Exception: + log.warning( + 'Cannot watch for changes to data file "%s" because it is an invalid path' + % path + ) + + if have_watchdog and not self._force_polling: + return _WatchdogAutoUpdaterV2(resolved_paths, self._on_file_change) + else: + return _PollingAutoUpdaterV2( + resolved_paths, self._on_file_change, self._poll_interval + ) + + def _on_file_change(self): + """ + Callback invoked when files change. + + Reloads all files and queues an update. + """ + with self._lock: + if self._closed: + return + + try: + # Reload all files + result = self._load_all_to_changeset() + + if isinstance(result, _Fail): + # Queue an error update + error_update = Update( + state=DataSourceState.INTERRUPTED, + error=DataSourceErrorInfo( + kind=DataSourceErrorKind.INVALID_DATA, + status_code=0, + time=current_time_millis(), + message=result.error, + ), + ) + self._update_queue.put(error_update) + else: + # Queue a successful update + update = Update( + state=DataSourceState.VALID, change_set=result.value + ) + self._update_queue.put(update) + + except Exception as e: + log.error("Error processing file change: %s" % repr(e)) + traceback.print_exc() + error_update = Update( + state=DataSourceState.INTERRUPTED, + error=DataSourceErrorInfo( + kind=DataSourceErrorKind.UNKNOWN, + status_code=0, + time=current_time_millis(), + message=f"Error processing file change: {str(e)}", + ), + ) + self._update_queue.put(error_update) + + +# Watch for changes to data files using the watchdog package. This uses native OS filesystem notifications +# if available for the current platform. +class _WatchdogAutoUpdaterV2: + def __init__(self, resolved_paths, on_change_callback): + watched_files = set(resolved_paths) + + class LDWatchdogHandler(watchdog.events.FileSystemEventHandler): + def on_any_event(self, event): + if event.src_path in watched_files: + on_change_callback() + + dir_paths = set() + for path in resolved_paths: + dir_paths.add(os.path.dirname(path)) + + self._observer = watchdog.observers.Observer() + handler = LDWatchdogHandler() + for path in dir_paths: + self._observer.schedule(handler, path) + self._observer.start() + + def stop(self): + self._observer.stop() + self._observer.join() + + +# Watch for changes to data files by polling their modification times. This is used if auto-update is +# on but the watchdog package is not installed. +class _PollingAutoUpdaterV2: + def __init__(self, resolved_paths, on_change_callback, interval): + self._paths = resolved_paths + self._on_change = on_change_callback + self._file_times = self._check_file_times() + self._timer = RepeatingTask( + "ldclient.datasource.filev2.poll", interval, interval, self._poll + ) + self._timer.start() + + def stop(self): + self._timer.stop() + + def _poll(self): + new_times = self._check_file_times() + changed = False + for file_path, file_time in self._file_times.items(): + if ( + new_times.get(file_path) is not None + and new_times.get(file_path) != file_time + ): + changed = True + break + self._file_times = new_times + if changed: + self._on_change() + + def _check_file_times(self): + ret = {} + for path in self._paths: + try: + ret[path] = os.path.getmtime(path) + except Exception: + log.warning( + "Failed to get modification time for %s. Setting to None", path + ) + ret[path] = None + return ret diff --git a/ldclient/impl/integrations/test_datav2/test_data_sourcev2.py b/ldclient/impl/integrations/test_datav2/test_data_sourcev2.py index 12f68c92..5e5b90d6 100644 --- a/ldclient/impl/integrations/test_datav2/test_data_sourcev2.py +++ b/ldclient/impl/integrations/test_datav2/test_data_sourcev2.py @@ -2,19 +2,19 @@ from queue import Empty, Queue from typing import Generator -from ldclient.impl.datasystem import BasisResult, Update -from ldclient.impl.datasystem.protocolv2 import ( - Basis, - ChangeSetBuilder, - IntentCode, - ObjectKind, - Selector -) from ldclient.impl.util import _Fail, _Success, current_time_millis from ldclient.interfaces import ( + Basis, + BasisResult, + ChangeSetBuilder, DataSourceErrorInfo, DataSourceErrorKind, - DataSourceState + DataSourceState, + IntentCode, + ObjectKind, + Selector, + SelectorStore, + Update ) @@ -42,7 +42,12 @@ def __init__(self, test_data): # - Added to `upsert_flag` to address potential race conditions. # - The `sync` method relies on Queue's thread-safe properties for updates. - def fetch(self) -> BasisResult: + @property + def name(self) -> str: + """Return the name of this data source.""" + return "TestDataV2" + + def fetch(self, ss: SelectorStore) -> BasisResult: """ Implementation of the Initializer.fetch method. @@ -64,28 +69,21 @@ def fetch(self) -> BasisResult: # Add all flags to the changeset for key, flag_data in init_data.items(): builder.add_put( - ObjectKind.FLAG, - key, - flag_data.get('version', 1), - flag_data + ObjectKind.FLAG, key, flag_data.get("version", 1), flag_data ) # Create selector for this version selector = Selector.new_selector(str(version), version) change_set = builder.finish(selector) - basis = Basis( - change_set=change_set, - persist=False, - environment_id=None - ) + basis = Basis(change_set=change_set, persist=False, environment_id=None) return _Success(basis) except Exception as e: return _Fail(f"Error fetching test data: {str(e)}") - def sync(self) -> Generator[Update, None, None]: + def sync(self, ss: SelectorStore) -> Generator[Update, None, None]: """ Implementation of the Synchronizer.sync method. @@ -93,7 +91,7 @@ def sync(self) -> Generator[Update, None, None]: """ # First yield initial data - initial_result = self.fetch() + initial_result = self.fetch(ss) if isinstance(initial_result, _Fail): yield Update( state=DataSourceState.OFF, @@ -101,15 +99,14 @@ def sync(self) -> Generator[Update, None, None]: kind=DataSourceErrorKind.STORE_ERROR, status_code=0, time=current_time_millis(), - message=initial_result.error - ) + message=initial_result.error, + ), ) return # Yield the initial successful state yield Update( - state=DataSourceState.VALID, - change_set=initial_result.value.change_set + state=DataSourceState.VALID, change_set=initial_result.value.change_set ) # Continue yielding updates as they arrive @@ -133,13 +130,13 @@ def sync(self) -> Generator[Update, None, None]: kind=DataSourceErrorKind.UNKNOWN, status_code=0, time=current_time_millis(), - message=f"Error in test data synchronizer: {str(e)}" - ) + message=f"Error in test data synchronizer: {str(e)}", + ), ) break - def close(self): - """Close the data source and clean up resources.""" + def stop(self): + """Stop the data source and clean up resources""" with self._lock: if self._closed: return @@ -170,9 +167,9 @@ def upsert_flag(self, flag_data: dict): # Add the updated flag builder.add_put( ObjectKind.FLAG, - flag_data['key'], - flag_data.get('version', 1), - flag_data + flag_data["key"], + flag_data.get("version", 1), + flag_data, ) # Create selector for this version @@ -180,10 +177,7 @@ def upsert_flag(self, flag_data: dict): change_set = builder.finish(selector) # Queue the update - update = Update( - state=DataSourceState.VALID, - change_set=change_set - ) + update = Update(state=DataSourceState.VALID, change_set=change_set) self._update_queue.put(update) @@ -195,7 +189,7 @@ def upsert_flag(self, flag_data: dict): kind=DataSourceErrorKind.STORE_ERROR, status_code=0, time=current_time_millis(), - message=f"Error processing flag update: {str(e)}" - ) + message=f"Error processing flag update: {str(e)}", + ), ) self._update_queue.put(error_update) diff --git a/ldclient/impl/util.py b/ldclient/impl/util.py index e60feb9d..54caf9de 100644 --- a/ldclient/impl/util.py +++ b/ldclient/impl/util.py @@ -4,7 +4,7 @@ import time from dataclasses import dataclass from datetime import timedelta -from typing import Any, Generic, Optional, TypeVar, Union +from typing import Any, Dict, Generic, Mapping, Optional, TypeVar, Union from urllib.parse import urlparse, urlunparse from ldclient.impl.http import _base_headers @@ -35,6 +35,9 @@ def timedelta_millis(delta: timedelta) -> float: # Compiled regex pattern for valid characters in application values and SDK keys _VALID_CHARACTERS_REGEX = re.compile(r"[^a-zA-Z0-9._-]") +_LD_ENVID_HEADER = 'X-LD-EnvID' +_LD_FD_FALLBACK_HEADER = 'X-LD-FD-Fallback' + def validate_application_info(application: dict, logger: logging.Logger) -> dict: return { @@ -285,6 +288,7 @@ class _Success(Generic[T]): class _Fail(Generic[E]): error: E exception: Optional[Exception] = None + headers: Optional[Mapping[str, Any]] = None # TODO(breaking): Replace the above Result class with an improved generic diff --git a/ldclient/integrations/__init__.py b/ldclient/integrations/__init__.py index c78b4023..6ec31c7c 100644 --- a/ldclient/integrations/__init__.py +++ b/ldclient/integrations/__init__.py @@ -6,7 +6,7 @@ from threading import Event from typing import Any, Callable, Dict, List, Mapping, Optional -from ldclient.config import Config +from ldclient.config import Builder, Config from ldclient.feature_store import CacheConfig from ldclient.feature_store_helpers import CachingStoreWrapper from ldclient.impl.integrations.consul.consul_feature_store import ( @@ -19,6 +19,9 @@ _DynamoDBFeatureStoreCore ) from ldclient.impl.integrations.files.file_data_source import _FileDataSource +from ldclient.impl.integrations.files.file_data_sourcev2 import ( + _FileDataSourceV2 +) from ldclient.impl.integrations.redis.redis_big_segment_store import ( _RedisBigSegmentStore ) @@ -250,3 +253,62 @@ def new_data_source(paths: List[str], auto_update: bool = False, poll_interval: :return: an object (actually a lambda) to be stored in the ``update_processor_class`` configuration property """ return lambda config, store, ready: _FileDataSource(store, config.data_source_update_sink, ready, paths, auto_update, poll_interval, force_polling) + + @staticmethod + def new_data_source_v2(paths: List[str], poll_interval: float = 1, force_polling: bool = False) -> Builder[Any]: + """Provides a way to use local files as a source of feature flag state using the FDv2 protocol. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + + This returns a builder that can be used with the FDv2 data system configuration as both an + Initializer and a Synchronizer. When used as an Initializer, it reads files once. When used + as a Synchronizer, it watches for file changes and automatically updates when files are modified. + + To use this component with the FDv2 data system, call ``new_data_source_v2`` and use the returned + builder with the custom data system configuration: + :: + + from ldclient.integrations import Files + from ldclient.impl.datasystem.config import custom + + file_source = Files.new_data_source_v2(paths=['my_flags.json']) + + # Use as initializer only + data_system = custom().initializers([file_source]).build() + config = Config(data_system=data_system) + + # Use as synchronizer only + data_system = custom().synchronizers(file_source).build() + config = Config(data_system=data_system) + + # Use as both initializer and synchronizer + data_system = custom().initializers([file_source]).synchronizers(file_source).build() + config = Config(data_system=data_system) + + This will cause the client not to connect to LaunchDarkly to get feature flags. The + client may still make network connections to send analytics events, unless you have disabled + this in your configuration with ``send_events`` or ``offline``. + + The format of the data files is the same as for the v1 file data source, described in the + SDK Reference Guide on `Reading flags from a file `_. + Note that in order to use YAML, you will need to install the ``pyyaml`` package. + + If the data source encounters any error in any file-- malformed content, a missing file, or a + duplicate key-- it will not load flags from any of the files. + + :param paths: the paths of the source files for loading flag data. These may be absolute paths + or relative to the current working directory. Files will be parsed as JSON unless the ``pyyaml`` + package is installed, in which case YAML is also allowed. + :param poll_interval: (default: 1) the minimum interval, in seconds, between checks for file + modifications when used as a Synchronizer. Only applies if the native file-watching mechanism + from ``watchdog`` is not being used. + :param force_polling: (default: false) True if the data source should implement file watching via + polling the filesystem even if a native mechanism is available. This is mainly for SDK testing. + + :return: a builder function that creates the file data source + """ + return lambda config: _FileDataSourceV2(paths, poll_interval, force_polling) diff --git a/ldclient/integrations/test_datav2.py b/ldclient/integrations/test_datav2.py index 84ccf30d..a2da52db 100644 --- a/ldclient/integrations/test_datav2.py +++ b/ldclient/integrations/test_datav2.py @@ -3,6 +3,7 @@ import copy from typing import Any, Dict, List, Optional, Set, Union +from ldclient.config import Config from ldclient.context import Context from ldclient.impl.integrations.test_datav2.test_data_sourcev2 import ( _TestDataSourceV2 @@ -550,17 +551,21 @@ class TestDataV2: :: from ldclient.impl.datasystem import config as datasystem_config + from ldclient.integrations.test_datav2 import TestDataV2 + td = TestDataV2.data_source() td.update(td.flag('flag-key-1').variation_for_all(True)) # Configure the data system with TestDataV2 as both initializer and synchronizer data_config = datasystem_config.custom() - data_config.initializers([lambda: td.build_initializer()]) - data_config.synchronizers(lambda: td.build_synchronizer()) + data_config.initializers([td.build_initializer]) + data_config.synchronizers(td.build_synchronizer) - # TODO(fdv2): This will be integrated with the main Config in a future version - # For now, TestDataV2 is primarily intended for unit testing scenarios + config = Config( + sdk_key, + datasystem_config=data_config.build(), + ) # flags can be updated at any time: td.update(td.flag('flag-key-1'). @@ -693,7 +698,7 @@ def _add_instance(self, instance): finally: self._lock.unlock() - def build_initializer(self) -> _TestDataSourceV2: + def build_initializer(self, _: Config) -> _TestDataSourceV2: """ Creates an initializer that can be used with the FDv2 data system. @@ -701,7 +706,7 @@ def build_initializer(self) -> _TestDataSourceV2: """ return _TestDataSourceV2(self) - def build_synchronizer(self) -> _TestDataSourceV2: + def build_synchronizer(self, _: Config) -> _TestDataSourceV2: """ Creates a synchronizer that can be used with the FDv2 data system. diff --git a/ldclient/interfaces.py b/ldclient/interfaces.py index 86a023fa..7a030d30 100644 --- a/ldclient/interfaces.py +++ b/ldclient/interfaces.py @@ -3,17 +3,60 @@ They may be useful in writing new implementations of these components, or for testing. """ - from abc import ABCMeta, abstractmethod, abstractproperty +from dataclasses import dataclass from enum import Enum -from typing import Any, Callable, Mapping, Optional +from typing import Any, Callable, Generator, List, Mapping, Optional, Protocol from ldclient.context import Context from ldclient.impl.listeners import Listeners +from ldclient.impl.util import _Result from .versioned_data_kind import VersionedDataKind +class DataStoreMode(Enum): + """ + DataStoreMode represents the mode of operation of a Data Store in FDV2 + mode. + + This enum is not stable, and not subject to any backwards compatibility + guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + READ_ONLY = 'read-only' + """ + READ_ONLY indicates that the data store is read-only. Data will never be + written back to the store by the SDK. + """ + + READ_WRITE = 'read-write' + """ + READ_WRITE indicates that the data store is read-write. Data from + initializers/synchronizers may be written to the store as necessary. + """ + + +class ReadOnlyStore(Protocol): + """ReadOnlyStore is a read-only interface for a feature store.""" + + @abstractmethod + def get(self, kind: VersionedDataKind, key: str, callback: Callable[[Any], Any] = lambda x: x) -> Any: + raise NotImplementedError + + @abstractmethod + def all(self, kind: VersionedDataKind, callback: Callable[[Any], Any] = lambda x: x) -> Any: + raise NotImplementedError + + @property + @abstractmethod + def initialized(self) -> bool: + raise NotImplementedError + + class FeatureStore: """ Interface for a versioned store for feature flags and related objects received from LaunchDarkly. @@ -923,8 +966,8 @@ class DataStoreStatus: __metaclass__ = ABCMeta def __init__(self, available: bool, stale: bool): - self.__available = available - self.__stale = stale + self._available = available + self._stale = stale @property def available(self) -> bool: @@ -939,7 +982,7 @@ def available(self) -> bool: :return: if store is available """ - return self.__available + return self._available @property def stale(self) -> bool: @@ -952,7 +995,18 @@ def stale(self) -> bool: :return: true if data should be rewritten """ - return self.__stale + return self._stale + + def __eq__(self, other): + """ + Ensures two instances of DataStoreStatus are the same if their properties are the same. + + :param other: The other instance to compare + :return: True if instances are equal, False otherwise + """ + if isinstance(other, DataStoreStatus): + return self._available == other._available and self._stale == other._stale + return False class DataStoreUpdateSink: @@ -1062,3 +1116,557 @@ def remove_listener(self, listener: Callable[[DataStoreStatus], None]): :param listener: the listener to remove; if no such listener was added, this does nothing """ + + +class EventName(str, Enum): + """ + EventName represents the name of an event that can be sent by the server for FDv2. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + PUT_OBJECT = "put-object" + """ + Specifies that an object should be added to the data set with upsert semantics. + """ + + DELETE_OBJECT = "delete-object" + """ + Specifies that an object should be removed from the data set. + """ + + SERVER_INTENT = "server-intent" + """ + Specifies the server's intent. + """ + + PAYLOAD_TRANSFERRED = "payload-transferred" + """ + Specifies that that all data required to bring the existing data set to + a new version has been transferred. + """ + + HEARTBEAT = "heart-beat" + """ + Keeps the connection alive. + """ + + GOODBYE = "goodbye" + """ + Specifies that the server is about to close the connection. + """ + + ERROR = "error" + """ + Specifies that an error occurred while serving the connection. + """ + + +@dataclass(frozen=True) +class Selector: + """ + Selector represents a particular snapshot of data. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + state: str = "" + version: int = 0 + + @staticmethod + def no_selector() -> "Selector": + """ + Returns an empty Selector. + """ + return Selector() + + def is_defined(self) -> bool: + """ + Returns True if the Selector has a value. + """ + return self != Selector.no_selector() + + def name(self) -> str: + """ + Event method. + """ + return EventName.PAYLOAD_TRANSFERRED + + @staticmethod + def new_selector(state: str, version: int) -> "Selector": + """ + Creates a new Selector from a state string and version. + """ + return Selector(state=state, version=version) + + def to_dict(self) -> dict: + """ + Serializes the Selector to a JSON-compatible dictionary. + """ + return {"state": self.state, "version": self.version} + + @staticmethod + def from_dict(data: dict) -> "Selector": + """ + Deserializes a Selector from a JSON-compatible dictionary. + """ + state = data.get("state") + version = data.get("version") + + if state is None or version is None: + raise ValueError("Missing required fields in Selector JSON.") + + return Selector(state=state, version=version) + + +class ChangeType(Enum): + """ + ChangeType specifies if an object is being upserted or deleted. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + PUT = "put" + """ + Represents an object being upserted. + """ + + DELETE = "delete" + """ + Represents an object being deleted. + """ + + +class ObjectKind(str, Enum): + """ + ObjectKind represents the kind of object. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + FLAG = "flag" + SEGMENT = "segment" + + +@dataclass(frozen=True) +class Change: + """ + Change represents a change to a piece of data, such as an update or deletion. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + action: ChangeType + kind: ObjectKind + key: str + version: int + object: Optional[dict] = None + + +class IntentCode(str, Enum): + """ + IntentCode represents the various intents that can be sent by the server. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + TRANSFER_FULL = "xfer-full" + """ + The server intends to send a full data set. + """ + TRANSFER_CHANGES = "xfer-changes" + """ + The server intends to send only the necessary changes to bring an existing + data set up-to-date. + """ + + TRANSFER_NONE = "none" + """ + The server intends to send no data (payload is up to date). + """ + + +@dataclass(frozen=True) +class ChangeSet: + """ + ChangeSet represents a list of changes to be applied. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + intent_code: IntentCode + changes: List[Change] + selector: Optional[Selector] + + +@dataclass(frozen=True) +class Basis: + """ + Basis represents the initial payload of data that a data source can + provide. Initializers provide this via fetch, whereas Synchronizers provide + it asynchronously. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + change_set: ChangeSet + persist: bool + environment_id: Optional[str] = None + + +class ChangeSetBuilder: + """ + ChangeSetBuilder is a helper for constructing a ChangeSet. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + def __init__(self): + """ + Initializes a new ChangeSetBuilder. + """ + self.intent = None + self.changes = [] + + @staticmethod + def no_changes() -> "ChangeSet": + """ + Represents an intent that the current data is up-to-date and doesn't + require changes. + """ + return ChangeSet( + intent_code=IntentCode.TRANSFER_NONE, selector=None, changes=[] + ) + + @staticmethod + def empty(selector) -> "ChangeSet": + """ + Returns an empty ChangeSet, which is useful for initializing a client + without data or for clearing out all existing data. + """ + return ChangeSet( + intent_code=IntentCode.TRANSFER_FULL, selector=selector, changes=[] + ) + + def start(self, intent: IntentCode): + """ + Begins a new change set with a given intent. + """ + self.intent = intent + self.changes = [] + + def expect_changes(self): + """ + Ensures that the current ChangeSetBuilder is prepared to handle changes. + + If a data source's initial connection reflects an updated status, we + need to keep the provided server intent. This allows subsequent changes + to come down the line without an explicit server intent. + + However, to maintain logical consistency, we need to ensure that the intent + is set to IntentTransferChanges. + """ + if self.intent is None: + raise ValueError("changeset: cannot expect changes without a server-intent") + + if self.intent != IntentCode.TRANSFER_NONE: + return + + self.intent = IntentCode.TRANSFER_CHANGES + + def reset(self): + """ + Clears any existing changes while preserving the current intent. + """ + self.changes = [] + + def finish(self, selector) -> ChangeSet: + """ + Identifies a changeset with a selector and returns the completed + changeset. Clears any existing changes while preserving the current + intent, so the builder can be reused. + """ + if self.intent is None: + raise ValueError("changeset: cannot complete without a server-intent") + + changeset = ChangeSet( + intent_code=self.intent, selector=selector, changes=self.changes + ) + self.changes = [] + + # Once a full transfer has been processed, all future changes should be + # assumed to be changes. Flag delivery can override this behavior by + # sending a new server intent to any connected stream. + if self.intent == IntentCode.TRANSFER_FULL: + self.intent = IntentCode.TRANSFER_CHANGES + + return changeset + + def add_put(self, kind, key, version, obj): + """ + Adds a new object to the changeset. + """ + self.changes.append( + Change( + action=ChangeType.PUT, kind=kind, key=key, version=version, object=obj + ) + ) + + def add_delete(self, kind, key, version): + """ + Adds a deletion to the changeset. + """ + self.changes.append( + Change(action=ChangeType.DELETE, kind=kind, key=key, version=version) + ) + + +@dataclass(frozen=True) +class Payload: + """ + Payload represents a payload delivered in a streaming response. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + id: str + target: int + code: IntentCode + reason: str + + def to_dict(self) -> dict: + """ + Serializes the Payload to a JSON-compatible dictionary. + """ + return { + "id": self.id, + "target": self.target, + "intentCode": self.code.value, + "reason": self.reason, + } + + @staticmethod + def from_dict(data: dict) -> "Payload": + """ + Create a Payload from a dictionary representation. + """ + intent_code = data.get("intentCode") + + if intent_code is None or not isinstance(intent_code, str): + raise ValueError( + "Invalid data for Payload: 'intentCode' key is missing or not a string" + ) + + return Payload( + id=data.get("id", ""), + target=data.get("target", 0), + code=IntentCode(intent_code), + reason=data.get("reason", ""), + ) + + +@dataclass(frozen=True) +class ServerIntent: + """ + ServerIntent represents the type of change associated with the payload + (e.g., transfer full, transfer changes, etc.) + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + payload: Payload + + def to_dict(self) -> dict: + """ + Serializes the ServerIntent to a JSON-compatible dictionary. + """ + return { + "payloads": [self.payload.to_dict()], + } + + @staticmethod + def from_dict(data: dict) -> "ServerIntent": + """ + Create a ServerIntent from a dictionary representation. + """ + if "payloads" not in data or not isinstance(data["payloads"], list): + raise ValueError( + "Invalid data for ServerIntent: 'payloads' key is missing or not a list" + ) + if len(data["payloads"]) != 1: + raise ValueError( + "Invalid data for ServerIntent: expected exactly one payload" + ) + + payload = data["payloads"][0] + if not isinstance(payload, dict): + raise ValueError("Invalid payload in ServerIntent: expected a dictionary") + + return ServerIntent(payload=Payload.from_dict(payload)) + + +class SelectorStore(Protocol): + """ + SelectorStore represents a component capable of providing Selectors + for data retrieval. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + @abstractmethod + def selector(self) -> Selector: + """ + get_selector should return a Selector object that defines the criteria + for data retrieval. + """ + raise NotImplementedError + + +BasisResult = _Result[Basis, str] + + +class Initializer(Protocol): # pylint: disable=too-few-public-methods + """ + Initializer represents a component capable of retrieving a single data + result, such as from the LD polling API. + + The intent of initializers is to quickly fetch an initial set of data, + which may be stale but is fast to retrieve. This initial data serves as a + foundation for a Synchronizer to build upon, enabling it to provide updates + as new changes occur. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + @property + @abstractmethod + def name(self) -> str: + """ + Returns the name of the initializer, which is used for logging and debugging. + """ + raise NotImplementedError + + @abstractmethod + def fetch(self, ss: SelectorStore) -> BasisResult: + """ + fetch should retrieve the initial data set for the data source, returning + a Basis object on success, or an error message on failure. + + :param ss: A SelectorStore that provides the Selector to use as a basis for data retrieval. + """ + raise NotImplementedError + + +@dataclass(frozen=True) +class Update: + """ + Update represents the results of a synchronizer's ongoing sync + method. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + + state: DataSourceState + change_set: Optional[ChangeSet] = None + error: Optional[DataSourceErrorInfo] = None + revert_to_fdv1: bool = False + environment_id: Optional[str] = None + + +class Synchronizer(Protocol): # pylint: disable=too-few-public-methods + """ + Synchronizer represents a component capable of synchronizing data from an external + data source, such as a streaming or polling API. + + It is responsible for yielding Update objects that represent the current state + of the data source, including any changes that have occurred since the last + synchronization. + + This type is not stable, and not subject to any backwards + compatibility guarantees or semantic versioning. It is not suitable for production usage. + + Do not use it. + You have been warned. + """ + @property + @abstractmethod + def name(self) -> str: + """ + Returns the name of the synchronizer, which is used for logging and debugging. + """ + raise NotImplementedError + + @abstractmethod + def sync(self, ss: SelectorStore) -> Generator[Update, None, None]: + """ + sync should begin the synchronization process for the data source, yielding + Update objects until the connection is closed or an unrecoverable error + occurs. + + :param ss: A SelectorStore that provides the Selector to use as a basis for data retrieval. + """ + raise NotImplementedError + + @abstractmethod + def stop(self): + """ + stop should halt the synchronization process, causing the sync method + to exit as soon as possible. + """ + raise NotImplementedError diff --git a/ldclient/testing/impl/datasourcev2/test_polling_initializer.py b/ldclient/testing/impl/datasourcev2/test_polling_initializer.py index be2e538f..bf152021 100644 --- a/ldclient/testing/impl/datasourcev2/test_polling_initializer.py +++ b/ldclient/testing/impl/datasourcev2/test_polling_initializer.py @@ -9,8 +9,9 @@ Selector, polling_payload_to_changeset ) -from ldclient.impl.datasystem.protocolv2 import ChangeSetBuilder, IntentCode from ldclient.impl.util import UnsuccessfulResponseException, _Fail, _Success +from ldclient.interfaces import ChangeSetBuilder, IntentCode +from ldclient.testing.mock_components import MockSelectorStore class MockExceptionThrowingPollingRequester: # pylint: disable=too-few-public-methods @@ -30,14 +31,14 @@ def test_polling_has_a_name(): mock_requester = MockPollingRequester(_Fail(error="failure message")) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - assert ds.name() == "PollingDataSourceV2" + assert ds.name == "PollingDataSourceV2" def test_error_is_returned_on_failure(): mock_requester = MockPollingRequester(_Fail(error="failure message")) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Fail) assert result.error == "failure message" @@ -50,7 +51,7 @@ def test_error_is_recoverable(): ) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Fail) assert result.error is not None @@ -64,7 +65,7 @@ def test_error_is_unrecoverable(): ) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Fail) assert result.error is not None @@ -78,7 +79,7 @@ def test_handles_transfer_none(): ) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Success) assert result.value is not None @@ -92,7 +93,7 @@ def test_handles_uncaught_exception(): mock_requester = MockExceptionThrowingPollingRequester() ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Fail) assert result.error is not None @@ -111,7 +112,7 @@ def test_handles_transfer_full(): mock_requester = MockPollingRequester(_Success(value=(change_set_result.value, {}))) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Success) assert result.value is not None @@ -129,7 +130,7 @@ def test_handles_transfer_changes(): mock_requester = MockPollingRequester(_Success(value=(change_set_result.value, {}))) ds = PollingDataSource(poll_interval=1.0, requester=mock_requester) - result = ds.fetch() + result = ds.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Success) assert result.value is not None diff --git a/ldclient/testing/impl/datasourcev2/test_polling_payload_parsing.py b/ldclient/testing/impl/datasourcev2/test_polling_payload_parsing.py index dae87706..580454f5 100644 --- a/ldclient/testing/impl/datasourcev2/test_polling_payload_parsing.py +++ b/ldclient/testing/impl/datasourcev2/test_polling_payload_parsing.py @@ -2,10 +2,11 @@ from ldclient.impl.datasourcev2.polling import ( IntentCode, + fdv1_polling_payload_to_changeset, polling_payload_to_changeset ) -from ldclient.impl.datasystem.protocolv2 import ChangeType, ObjectKind from ldclient.impl.util import _Fail, _Success +from ldclient.interfaces import ChangeType, ObjectKind def test_payload_is_missing_events_key(): @@ -151,3 +152,212 @@ def test_fails_if_starts_with_put(): assert ( result.exception.args[0] == "changeset: cannot complete without a server-intent" ) + + +# FDv1 Payload Parsing Tests +def test_fdv1_payload_empty_flags_and_segments(): + """Test that FDv1 payload with empty flags and segments produces empty changeset.""" + data = { + "flags": {}, + "segments": {} + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert change_set.intent_code == IntentCode.TRANSFER_FULL + assert len(change_set.changes) == 0 + # FDv1 doesn't use selectors + assert change_set.selector is not None + assert not change_set.selector.is_defined() + + +def test_fdv1_payload_with_single_flag(): + """Test that FDv1 payload with a single flag is parsed correctly.""" + data = { + "flags": { + "test-flag": { + "key": "test-flag", + "version": 1, + "on": True, + "variations": [True, False] + } + }, + "segments": {} + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert change_set.intent_code == IntentCode.TRANSFER_FULL + assert len(change_set.changes) == 1 + + change = change_set.changes[0] + assert change.action == ChangeType.PUT + assert change.kind == ObjectKind.FLAG + assert change.key == "test-flag" + assert change.version == 1 + + +def test_fdv1_payload_with_multiple_flags(): + """Test that FDv1 payload with multiple flags is parsed correctly.""" + data = { + "flags": { + "flag-1": {"key": "flag-1", "version": 1, "on": True}, + "flag-2": {"key": "flag-2", "version": 2, "on": False}, + "flag-3": {"key": "flag-3", "version": 3, "on": True} + }, + "segments": {} + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert len(change_set.changes) == 3 + + flag_keys = {c.key for c in change_set.changes} + assert flag_keys == {"flag-1", "flag-2", "flag-3"} + + +def test_fdv1_payload_with_single_segment(): + """Test that FDv1 payload with a single segment is parsed correctly.""" + data = { + "flags": {}, + "segments": { + "test-segment": { + "key": "test-segment", + "version": 5, + "included": ["user1", "user2"] + } + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert len(change_set.changes) == 1 + + change = change_set.changes[0] + assert change.action == ChangeType.PUT + assert change.kind == ObjectKind.SEGMENT + assert change.key == "test-segment" + assert change.version == 5 + + +def test_fdv1_payload_with_flags_and_segments(): + """Test that FDv1 payload with both flags and segments is parsed correctly.""" + data = { + "flags": { + "flag-1": {"key": "flag-1", "version": 1, "on": True}, + "flag-2": {"key": "flag-2", "version": 2, "on": False} + }, + "segments": { + "segment-1": {"key": "segment-1", "version": 10}, + "segment-2": {"key": "segment-2", "version": 20} + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert len(change_set.changes) == 4 + + flag_changes = [c for c in change_set.changes if c.kind == ObjectKind.FLAG] + segment_changes = [c for c in change_set.changes if c.kind == ObjectKind.SEGMENT] + + assert len(flag_changes) == 2 + assert len(segment_changes) == 2 + + +def test_fdv1_payload_flags_not_dict(): + """Test that FDv1 payload parser fails when flags namespace is not a dict.""" + data = { + "flags": "not a dict" + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Fail) + assert "not a dictionary" in result.error + + +def test_fdv1_payload_segments_not_dict(): + """Test that FDv1 payload parser fails when segments namespace is not a dict.""" + data = { + "flags": {}, + "segments": "not a dict" + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Fail) + assert "not a dictionary" in result.error + + +def test_fdv1_payload_flag_value_not_dict(): + """Test that FDv1 payload parser fails when a flag value is not a dict.""" + data = { + "flags": { + "bad-flag": "not a dict" + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Fail) + assert "not a dictionary" in result.error + + +def test_fdv1_payload_flag_missing_version(): + """Test that FDv1 payload parser fails when a flag is missing version.""" + data = { + "flags": { + "no-version-flag": { + "key": "no-version-flag", + "on": True + } + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Fail) + assert "does not have a version set" in result.error + + +def test_fdv1_payload_segment_missing_version(): + """Test that FDv1 payload parser fails when a segment is missing version.""" + data = { + "flags": {}, + "segments": { + "no-version-segment": { + "key": "no-version-segment", + "included": [] + } + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Fail) + assert "does not have a version set" in result.error + + +def test_fdv1_payload_only_flags_no_segments_key(): + """Test that FDv1 payload works when segments key is missing entirely.""" + data = { + "flags": { + "test-flag": {"key": "test-flag", "version": 1, "on": True} + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert len(change_set.changes) == 1 + assert change_set.changes[0].key == "test-flag" + + +def test_fdv1_payload_only_segments_no_flags_key(): + """Test that FDv1 payload works when flags key is missing entirely.""" + data = { + "segments": { + "test-segment": {"key": "test-segment", "version": 1} + } + } + result = fdv1_polling_payload_to_changeset(data) + assert isinstance(result, _Success) + + change_set = result.value + assert len(change_set.changes) == 1 + assert change_set.changes[0].key == "test-segment" diff --git a/ldclient/testing/impl/datasourcev2/test_polling_synchronizer.py b/ldclient/testing/impl/datasourcev2/test_polling_synchronizer.py index ff8bf2eb..ebb2674a 100644 --- a/ldclient/testing/impl/datasourcev2/test_polling_synchronizer.py +++ b/ldclient/testing/impl/datasourcev2/test_polling_synchronizer.py @@ -4,24 +4,33 @@ import pytest from ld_eventsource.sse_client import Event -from ldclient.impl.datasourcev2 import PollingResult -from ldclient.impl.datasourcev2.polling import PollingDataSource +from ldclient.impl.datasourcev2.polling import PollingDataSource, PollingResult from ldclient.impl.datasystem.protocolv2 import ( - ChangeSetBuilder, - ChangeType, DeleteObject, Error, EventName, Goodbye, + PutObject +) +from ldclient.impl.util import ( + _LD_ENVID_HEADER, + _LD_FD_FALLBACK_HEADER, + UnsuccessfulResponseException, + _Fail, + _Success +) +from ldclient.interfaces import ( + ChangeSetBuilder, + ChangeType, + DataSourceErrorKind, + DataSourceState, IntentCode, ObjectKind, Payload, - PutObject, Selector, ServerIntent ) -from ldclient.impl.util import UnsuccessfulResponseException, _Fail, _Success -from ldclient.interfaces import DataSourceErrorKind, DataSourceState +from ldclient.testing.mock_components import MockSelectorStore class ListBasedRequester: @@ -103,7 +112,7 @@ def test_handles_no_changes(): poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) ) - valid = next(synchronizer.sync()) + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert valid.state == DataSourceState.VALID assert valid.error is None @@ -124,7 +133,7 @@ def test_handles_empty_changeset(): synchronizer = PollingDataSource( poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) ) - valid = next(synchronizer.sync()) + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert valid.state == DataSourceState.VALID assert valid.error is None @@ -152,7 +161,7 @@ def test_handles_put_objects(): synchronizer = PollingDataSource( poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) ) - valid = next(synchronizer.sync()) + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert valid.state == DataSourceState.VALID assert valid.error is None @@ -183,7 +192,7 @@ def test_handles_delete_objects(): synchronizer = PollingDataSource( poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) ) - valid = next(synchronizer.sync()) + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert valid.state == DataSourceState.VALID assert valid.error is None @@ -202,86 +211,6 @@ def test_handles_delete_objects(): assert valid.change_set.intent_code == IntentCode.TRANSFER_FULL -# def test_swallows_goodbye(events): # pylint: disable=redefined-outer-name -# builder = list_sse_client( -# [ -# events[EventName.SERVER_INTENT], -# events[EventName.GOODBYE], -# events[EventName.PAYLOAD_TRANSFERRED], -# ] -# ) -# -# synchronizer = StreamingSynchronizer(Config(sdk_key="key"), builder) -# updates = list(synchronizer.sync()) -# -# builder = ChangeSetBuilder() -# builder.start(intent=IntentCode.TRANSFER_FULL) -# change_set = builder.finish(selector=Selector(state="p:SOMETHING:300", version=300)) -# headers = {} -# polling_result: PollingResult = _Success(value=(change_set, headers)) -# -# synchronizer = PollingDataSource( -# poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) -# ) -# updates = list(synchronizer.sync()) -# -# assert len(updates) == 1 -# assert updates[0].state == DataSourceState.VALID -# assert updates[0].error is None -# assert updates[0].revert_to_fdv1 is False -# assert updates[0].environment_id is None -# -# assert updates[0].change_set is not None -# assert len(updates[0].change_set.changes) == 1 -# assert updates[0].change_set.changes[0].action == ChangeType.DELETE -# assert updates[0].change_set.changes[0].kind == ObjectKind.FLAG -# assert updates[0].change_set.changes[0].key == "flag-key" -# assert updates[0].change_set.changes[0].version == 101 -# assert updates[0].change_set.selector is not None -# assert updates[0].change_set.selector.version == 300 -# assert updates[0].change_set.selector.state == "p:SOMETHING:300" -# assert updates[0].change_set.intent_code == IntentCode.TRANSFER_FULL -# -# assert len(updates) == 1 -# assert updates[0].state == DataSourceState.VALID -# assert updates[0].error is None -# assert updates[0].revert_to_fdv1 is False -# assert updates[0].environment_id is None -# -# assert updates[0].change_set is not None -# assert len(updates[0].change_set.changes) == 0 -# assert updates[0].change_set.selector is not None -# assert updates[0].change_set.selector.version == 300 -# assert updates[0].change_set.selector.state == "p:SOMETHING:300" -# assert updates[0].change_set.intent_code == IntentCode.TRANSFER_FULL -# -# -# def test_swallows_heartbeat(events): # pylint: disable=redefined-outer-name -# builder = list_sse_client( -# [ -# events[EventName.SERVER_INTENT], -# events[EventName.HEARTBEAT], -# events[EventName.PAYLOAD_TRANSFERRED], -# ] -# ) -# -# synchronizer = StreamingSynchronizer(Config(sdk_key="key"), builder) -# updates = list(synchronizer.sync()) -# -# assert len(updates) == 1 -# assert updates[0].state == DataSourceState.VALID -# assert updates[0].error is None -# assert updates[0].revert_to_fdv1 is False -# assert updates[0].environment_id is None -# -# assert updates[0].change_set is not None -# assert len(updates[0].change_set.changes) == 0 -# assert updates[0].change_set.selector is not None -# assert updates[0].change_set.selector.version == 300 -# assert updates[0].change_set.selector.state == "p:SOMETHING:300" -# assert updates[0].change_set.intent_code == IntentCode.TRANSFER_FULL -# -# def test_generic_error_interrupts_and_recovers(): builder = ChangeSetBuilder() builder.start(intent=IntentCode.TRANSFER_FULL) @@ -296,7 +225,7 @@ def test_generic_error_interrupts_and_recovers(): results=iter([_Fail(error="error for test"), polling_result]) ), ) - sync = synchronizer.sync() + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) interrupted = next(sync) valid = next(sync) @@ -330,7 +259,7 @@ def test_recoverable_error_continues(): poll_interval=0.01, requester=ListBasedRequester(results=iter([_failure, polling_result])), ) - sync = synchronizer.sync() + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) interrupted = next(sync) valid = next(sync) @@ -368,7 +297,7 @@ def test_unrecoverable_error_shuts_down(): poll_interval=0.01, requester=ListBasedRequester(results=iter([_failure, polling_result])), ) - sync = synchronizer.sync() + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) off = next(sync) assert off.state == DataSourceState.OFF assert off.error is not None @@ -383,3 +312,169 @@ def test_unrecoverable_error_shuts_down(): assert False, "Expected StopIteration" except StopIteration: pass + + +def test_envid_from_success_headers(): + """Test that environment ID is captured from successful polling response headers""" + change_set = ChangeSetBuilder.no_changes() + headers = {_LD_ENVID_HEADER: 'test-env-polling-123'} + polling_result: PollingResult = _Success(value=(change_set, headers)) + + synchronizer = PollingDataSource( + poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) + ) + + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert valid.state == DataSourceState.VALID + assert valid.error is None + assert valid.revert_to_fdv1 is False + assert valid.environment_id == 'test-env-polling-123' + + +def test_envid_from_success_with_changeset(): + """Test that environment ID is captured from polling response with actual changes""" + builder = ChangeSetBuilder() + builder.start(intent=IntentCode.TRANSFER_FULL) + builder.add_put( + version=100, kind=ObjectKind.FLAG, key="flag-key", obj={"key": "flag-key"} + ) + change_set = builder.finish(selector=Selector(state="p:SOMETHING:300", version=300)) + headers = {_LD_ENVID_HEADER: 'test-env-456'} + polling_result: PollingResult = _Success(value=(change_set, headers)) + + synchronizer = PollingDataSource( + poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) + ) + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert valid.state == DataSourceState.VALID + assert valid.environment_id == 'test-env-456' + assert valid.change_set is not None + assert len(valid.change_set.changes) == 1 + + +def test_envid_from_fallback_headers(): + """Test that environment ID is captured when fallback header is present on success""" + change_set = ChangeSetBuilder.no_changes() + headers = { + _LD_ENVID_HEADER: 'test-env-fallback', + _LD_FD_FALLBACK_HEADER: 'true' + } + polling_result: PollingResult = _Success(value=(change_set, headers)) + + synchronizer = PollingDataSource( + poll_interval=0.01, requester=ListBasedRequester(results=iter([polling_result])) + ) + + valid = next(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert valid.state == DataSourceState.VALID + assert valid.revert_to_fdv1 is True + assert valid.environment_id == 'test-env-fallback' + + +def test_envid_from_error_headers_recoverable(): + """Test that environment ID is captured from error response headers for recoverable errors""" + builder = ChangeSetBuilder() + builder.start(intent=IntentCode.TRANSFER_FULL) + builder.add_delete(version=101, kind=ObjectKind.FLAG, key="flag-key") + change_set = builder.finish(selector=Selector(state="p:SOMETHING:300", version=300)) + headers_success = {_LD_ENVID_HEADER: 'test-env-success'} + polling_result: PollingResult = _Success(value=(change_set, headers_success)) + + headers_error = {_LD_ENVID_HEADER: 'test-env-408'} + _failure = _Fail( + error="error for test", + exception=UnsuccessfulResponseException(status=408), + headers=headers_error + ) + + synchronizer = PollingDataSource( + poll_interval=0.01, + requester=ListBasedRequester(results=iter([_failure, polling_result])), + ) + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) + interrupted = next(sync) + valid = next(sync) + + assert interrupted.state == DataSourceState.INTERRUPTED + assert interrupted.environment_id == 'test-env-408' + assert interrupted.error is not None + assert interrupted.error.status_code == 408 + + assert valid.state == DataSourceState.VALID + assert valid.environment_id == 'test-env-success' + + +def test_envid_from_error_headers_unrecoverable(): + """Test that environment ID is captured from error response headers for unrecoverable errors""" + headers_error = {_LD_ENVID_HEADER: 'test-env-401'} + _failure = _Fail( + error="error for test", + exception=UnsuccessfulResponseException(status=401), + headers=headers_error + ) + + synchronizer = PollingDataSource( + poll_interval=0.01, + requester=ListBasedRequester(results=iter([_failure])), + ) + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) + off = next(sync) + + assert off.state == DataSourceState.OFF + assert off.environment_id == 'test-env-401' + assert off.error is not None + assert off.error.status_code == 401 + + +def test_envid_from_error_with_fallback(): + """Test that environment ID and fallback are captured from error response""" + headers_error = { + _LD_ENVID_HEADER: 'test-env-503', + _LD_FD_FALLBACK_HEADER: 'true' + } + _failure = _Fail( + error="error for test", + exception=UnsuccessfulResponseException(status=503), + headers=headers_error + ) + + synchronizer = PollingDataSource( + poll_interval=0.01, + requester=ListBasedRequester(results=iter([_failure])), + ) + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) + off = next(sync) + + assert off.state == DataSourceState.OFF + assert off.revert_to_fdv1 is True + assert off.environment_id == 'test-env-503' + + +def test_envid_from_generic_error_with_headers(): + """Test that environment ID is captured from generic errors with headers""" + builder = ChangeSetBuilder() + builder.start(intent=IntentCode.TRANSFER_FULL) + change_set = builder.finish(selector=Selector(state="p:SOMETHING:300", version=300)) + headers_success = {} + polling_result: PollingResult = _Success(value=(change_set, headers_success)) + + headers_error = {_LD_ENVID_HEADER: 'test-env-generic'} + _failure = _Fail(error="generic error for test", headers=headers_error) + + synchronizer = PollingDataSource( + poll_interval=0.01, + requester=ListBasedRequester(results=iter([_failure, polling_result])), + ) + sync = synchronizer.sync(MockSelectorStore(Selector.no_selector())) + interrupted = next(sync) + valid = next(sync) + + assert interrupted.state == DataSourceState.INTERRUPTED + assert interrupted.environment_id == 'test-env-generic' + assert interrupted.error is not None + assert interrupted.error.kind == DataSourceErrorKind.NETWORK_ERROR + + assert valid.state == DataSourceState.VALID diff --git a/ldclient/testing/impl/datasourcev2/test_streaming_synchronizer.py b/ldclient/testing/impl/datasourcev2/test_streaming_synchronizer.py index 8aa66bbb..b91d5fba 100644 --- a/ldclient/testing/impl/datasourcev2/test_streaming_synchronizer.py +++ b/ldclient/testing/impl/datasourcev2/test_streaming_synchronizer.py @@ -6,7 +6,7 @@ from typing import Iterable, List, Optional import pytest -from ld_eventsource.actions import Action +from ld_eventsource.actions import Action, Start from ld_eventsource.http import HTTPStatusError from ld_eventsource.sse_client import Event, Fault @@ -17,25 +17,31 @@ StreamingDataSource ) from ldclient.impl.datasystem.protocolv2 import ( - ChangeType, DeleteObject, Error, EventName, Goodbye, + PutObject +) +from ldclient.impl.util import _LD_ENVID_HEADER, _LD_FD_FALLBACK_HEADER +from ldclient.interfaces import ( + ChangeType, + DataSourceErrorKind, + DataSourceState, IntentCode, ObjectKind, Payload, - PutObject, Selector, + SelectorStore, ServerIntent ) -from ldclient.interfaces import DataSourceErrorKind, DataSourceState +from ldclient.testing.mock_components import MockSelectorStore def list_sse_client( events: Iterable[Action], # pylint: disable=redefined-outer-name ) -> SseClientBuilder: - def builder(_: Config) -> SSEClient: + def builder(config: Config, ss: SelectorStore) -> SSEClient: return ListBasedSseClient(events) return builder @@ -51,6 +57,16 @@ def __init__( def all(self) -> Iterable[Action]: return self._events + @property + def next_retry_delay(self): + return 1 + + def interrupt(self): + pass + + def close(self): + pass + class HttpExceptionThrowingSseClient: def __init__(self, status_codes: List[int]): # pylint: disable=redefined-outer-name @@ -74,18 +90,18 @@ class UnknownTypeOfEvent(Action): pass unknown_named_event = Event(event="Unknown") - builder = list_sse_client([UnknownTypeOfEvent(), unknown_named_event]) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = list_sse_client([UnknownTypeOfEvent(), unknown_named_event]) - assert len(list(synchronizer.sync())) == 0 + assert len(list(synchronizer.sync(MockSelectorStore(Selector.no_selector())))) == 0 def test_ignores_faults_without_errors(): errorless_fault = Fault(error=None) - builder = list_sse_client([errorless_fault]) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = list_sse_client([errorless_fault]) - assert len(list(synchronizer.sync())) == 0 + assert len(list(synchronizer.sync(MockSelectorStore(Selector.no_selector())))) == 0 @pytest.fixture @@ -160,10 +176,10 @@ def test_handles_no_changes(): event=EventName.SERVER_INTENT, data=json.dumps(server_intent.to_dict()), ) - builder = list_sse_client([intent_event]) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = list_sse_client([intent_event]) + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -181,8 +197,9 @@ def test_handles_empty_changeset(events): # pylint: disable=redefined-outer-nam ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -207,8 +224,9 @@ def test_handles_put_objects(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -238,8 +256,9 @@ def test_handles_delete_objects(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -268,8 +287,9 @@ def test_swallows_goodbye(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -294,8 +314,9 @@ def test_swallows_heartbeat(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -322,8 +343,9 @@ def test_error_resets(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.VALID @@ -345,8 +367,9 @@ def test_handles_out_of_order(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.INTERRUPTED @@ -375,8 +398,9 @@ def test_invalid_json_decoding(events): # pylint: disable=redefined-outer-name ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 2 assert updates[0].state == DataSourceState.INTERRUPTED @@ -396,10 +420,12 @@ def test_invalid_json_decoding(events): # pylint: disable=redefined-outer-name def test_stops_on_unrecoverable_status_code( events, ): # pylint: disable=redefined-outer-name + error = HTTPStatusError(401) + fault = Fault(error=error) builder = list_sse_client( [ # This will generate an error but the stream should continue - Fault(error=HTTPStatusError(401)), + fault, # We send these valid combinations to ensure the stream is NOT # being processed after the 401. events[EventName.SERVER_INTENT], @@ -407,8 +433,9 @@ def test_stops_on_unrecoverable_status_code( ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 1 assert updates[0].state == DataSourceState.OFF @@ -424,20 +451,27 @@ def test_stops_on_unrecoverable_status_code( def test_continues_on_recoverable_status_code( events, ): # pylint: disable=redefined-outer-name + error1 = HTTPStatusError(400) + fault1 = Fault(error=error1) + + error2 = HTTPStatusError(408) + fault2 = Fault(error=error2) + builder = list_sse_client( [ # This will generate an error but the stream should continue - Fault(error=HTTPStatusError(400)), + fault1, events[EventName.SERVER_INTENT], - Fault(error=HTTPStatusError(408)), + fault2, # We send these valid combinations to ensure the stream will # continue to be processed. events[EventName.SERVER_INTENT], events[EventName.PAYLOAD_TRANSFERRED], ] ) - synchronizer = StreamingDataSource(Config(sdk_key="key"), builder) - updates = list(synchronizer.sync()) + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) assert len(updates) == 3 assert updates[0].state == DataSourceState.INTERRUPTED @@ -456,3 +490,207 @@ def test_continues_on_recoverable_status_code( assert updates[2].change_set.selector.version == 300 assert updates[2].change_set.selector.state == "p:SOMETHING:300" assert updates[2].change_set.intent_code == IntentCode.TRANSFER_FULL + + +def test_envid_from_start_action(events): # pylint: disable=redefined-outer-name + """Test that environment ID is captured from Start action headers""" + start_action = Start(headers={_LD_ENVID_HEADER: 'test-env-123'}) + + builder = list_sse_client( + [ + start_action, + events[EventName.SERVER_INTENT], + events[EventName.PAYLOAD_TRANSFERRED], + ] + ) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 1 + assert updates[0].state == DataSourceState.VALID + assert updates[0].environment_id == 'test-env-123' + + +def test_envid_not_cleared_from_next_start(events): # pylint: disable=redefined-outer-name + """Test that environment ID is captured from Start action headers""" + start_action_with_headers = Start(headers={_LD_ENVID_HEADER: 'test-env-123'}) + start_action_without_headers = Start() + + builder = list_sse_client( + [ + start_action_with_headers, + events[EventName.SERVER_INTENT], + events[EventName.PAYLOAD_TRANSFERRED], + start_action_without_headers, + events[EventName.SERVER_INTENT], + events[EventName.PAYLOAD_TRANSFERRED], + ] + ) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 2 + assert updates[0].state == DataSourceState.VALID + assert updates[0].environment_id == 'test-env-123' + + assert updates[1].state == DataSourceState.VALID + assert updates[1].environment_id == 'test-env-123' + + +def test_envid_preserved_across_events(events): # pylint: disable=redefined-outer-name + """Test that environment ID is preserved across multiple events after being set on Start""" + start_action = Start(headers={_LD_ENVID_HEADER: 'test-env-456'}) + + builder = list_sse_client( + [ + start_action, + events[EventName.SERVER_INTENT], + events[EventName.PUT_OBJECT], + events[EventName.PAYLOAD_TRANSFERRED], + ] + ) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 1 + assert updates[0].state == DataSourceState.VALID + assert updates[0].environment_id == 'test-env-456' + assert updates[0].change_set is not None + assert len(updates[0].change_set.changes) == 1 + + +def test_envid_from_fallback_header(): + """Test that environment ID is captured when fallback header is present""" + start_action = Start(headers={_LD_ENVID_HEADER: 'test-env-fallback', _LD_FD_FALLBACK_HEADER: 'true'}) + + builder = list_sse_client([start_action]) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 1 + assert updates[0].state == DataSourceState.OFF + assert updates[0].revert_to_fdv1 is True + assert updates[0].environment_id == 'test-env-fallback' + + +def test_envid_from_fault_action(): + """Test that environment ID is captured from Fault action headers""" + error = HTTPStatusError(401, headers={_LD_ENVID_HEADER: 'test-env-fault'}) + fault_action = Fault(error=error) + + builder = list_sse_client([fault_action]) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 1 + assert updates[0].state == DataSourceState.OFF + assert updates[0].environment_id == 'test-env-fault' + assert updates[0].error is not None + assert updates[0].error.status_code == 401 + + +def test_envid_not_cleared_from_next_error(): + """Test that environment ID is captured from Fault action headers""" + error_with_headers_ = HTTPStatusError(408, headers={_LD_ENVID_HEADER: 'test-env-fault'}) + error_without_headers_ = HTTPStatusError(401) + fault_action_with_headers = Fault(error=error_with_headers_) + fault_action_without_headers = Fault(error=error_without_headers_) + + builder = list_sse_client([fault_action_with_headers, fault_action_without_headers]) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 2 + assert updates[0].state == DataSourceState.INTERRUPTED + assert updates[0].environment_id == 'test-env-fault' + assert updates[0].error is not None + assert updates[0].error.status_code == 408 + + assert updates[1].state == DataSourceState.OFF + assert updates[1].environment_id == 'test-env-fault' + assert updates[1].error is not None + assert updates[1].error.status_code == 401 + + +def test_envid_from_fault_with_fallback(): + """Test that environment ID and fallback are captured from Fault action""" + error = HTTPStatusError(503, headers={_LD_ENVID_HEADER: 'test-env-503', _LD_FD_FALLBACK_HEADER: 'true'}) + fault_action = Fault(error=error) + + builder = list_sse_client([fault_action]) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 1 + assert updates[0].state == DataSourceState.OFF + assert updates[0].revert_to_fdv1 is True + assert updates[0].environment_id == 'test-env-503' + + +def test_envid_from_recoverable_fault(events): # pylint: disable=redefined-outer-name + """Test that environment ID is captured from recoverable Fault and preserved in subsequent events""" + error = HTTPStatusError(400, headers={_LD_ENVID_HEADER: 'test-env-400'}) + fault_action = Fault(error=error) + + builder = list_sse_client( + [ + fault_action, + events[EventName.SERVER_INTENT], + events[EventName.PAYLOAD_TRANSFERRED], + ] + ) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 2 + # First update from the fault + assert updates[0].state == DataSourceState.INTERRUPTED + assert updates[0].environment_id == 'test-env-400' + + # Second update should preserve the envid + assert updates[1].state == DataSourceState.VALID + assert updates[1].environment_id == 'test-env-400' + + +def test_envid_missing_when_no_headers(): + """Test that environment ID is None when no headers are present""" + start_action = Start() + + server_intent = ServerIntent( + payload=Payload( + id="id", + target=300, + code=IntentCode.TRANSFER_NONE, + reason="up-to-date", + ) + ) + intent_event = Event( + event=EventName.SERVER_INTENT, + data=json.dumps(server_intent.to_dict()), + ) + + builder = list_sse_client([start_action, intent_event]) + + synchronizer = StreamingDataSource(Config(sdk_key="key")) + synchronizer._sse_client_builder = builder + updates = list(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) + + assert len(updates) == 1 + assert updates[0].state == DataSourceState.VALID + assert updates[0].environment_id is None diff --git a/ldclient/testing/impl/datasystem/test_config.py b/ldclient/testing/impl/datasystem/test_config.py index c7c0925b..c9f14c31 100644 --- a/ldclient/testing/impl/datasystem/test_config.py +++ b/ldclient/testing/impl/datasystem/test_config.py @@ -1,12 +1,11 @@ import dataclasses -from unittest.mock import MagicMock, Mock +from unittest.mock import Mock import pytest from ldclient.config import Config as LDConfig -from ldclient.impl.datasystem import Initializer, Synchronizer -from ldclient.impl.datasystem.config import ( - Config, +from ldclient.config import DataSystemConfig +from ldclient.datasystem import ( ConfigBuilder, custom, default, @@ -63,31 +62,12 @@ def test_config_builder_build_success(): config = builder.build() - assert isinstance(config, Config) + assert isinstance(config, DataSystemConfig) assert config.initializers == [mock_initializer] assert config.primary_synchronizer == mock_primary assert config.secondary_synchronizer == mock_secondary -def test_config_builder_build_missing_primary_synchronizer(): - """Test that build fails when primary synchronizer is not set.""" - builder = ConfigBuilder() - - with pytest.raises(ValueError, match="Primary synchronizer must be set"): - builder.build() - - -def test_config_builder_build_with_initializers_only(): - """Test that build fails when only initializers are set.""" - builder = ConfigBuilder() - mock_initializer = Mock() - - builder.initializers([mock_initializer]) - - with pytest.raises(ValueError, match="Primary synchronizer must be set"): - builder.build() - - def test_config_builder_method_chaining(): """Test that all builder methods support method chaining.""" builder = ConfigBuilder() @@ -146,9 +126,7 @@ def test_custom_builder(): def test_default_config_builder(): """Test that default() returns a properly configured ConfigBuilder.""" - mock_ld_config = Mock(spec=LDConfig) - - builder = default(mock_ld_config) + builder = default() assert isinstance(builder, ConfigBuilder) # The actual implementation details would be tested in integration tests @@ -157,9 +135,7 @@ def test_default_config_builder(): def test_streaming_config_builder(): """Test that streaming() returns a properly configured ConfigBuilder.""" - mock_ld_config = Mock(spec=LDConfig) - - builder = streaming(mock_ld_config) + builder = streaming() assert isinstance(builder, ConfigBuilder) # The actual implementation details would be tested in integration tests @@ -168,9 +144,7 @@ def test_streaming_config_builder(): def test_polling_config_builder(): """Test that polling() returns a properly configured ConfigBuilder.""" - mock_ld_config = Mock(spec=LDConfig) - - builder = polling(mock_ld_config) + builder = polling() assert isinstance(builder, ConfigBuilder) # The actual implementation details would be tested in integration tests @@ -178,11 +152,11 @@ def test_polling_config_builder(): def test_config_dataclass_immutability(): - """Test that Config instances are immutable (frozen dataclass).""" + """Test that DataSystemConfig instances are immutable (frozen dataclass).""" mock_primary = Mock() mock_secondary = Mock() - config = Config( + config = DataSystemConfig( initializers=None, primary_synchronizer=mock_primary, secondary_synchronizer=mock_secondary, diff --git a/ldclient/testing/impl/datasystem/test_fdv2_datasystem.py b/ldclient/testing/impl/datasystem/test_fdv2_datasystem.py new file mode 100644 index 00000000..c49b7137 --- /dev/null +++ b/ldclient/testing/impl/datasystem/test_fdv2_datasystem.py @@ -0,0 +1,434 @@ +# pylint: disable=missing-docstring + +from threading import Event +from typing import List + +from mock import Mock + +from ldclient.config import Config, DataSystemConfig +from ldclient.impl.datasystem import DataAvailability +from ldclient.impl.datasystem.fdv2 import FDv2 +from ldclient.integrations.test_datav2 import TestDataV2 +from ldclient.interfaces import ( + DataSourceState, + DataSourceStatus, + FlagChange, + Synchronizer, + Update +) +from ldclient.versioned_data_kind import FEATURES + + +def test_two_phase_init(): + td_initializer = TestDataV2.data_source() + td_initializer.update(td_initializer.flag("feature-flag").on(True)) + + td_synchronizer = TestDataV2.data_source() + # Set this to true, and then to false to ensure the version number exceeded + # the initializer version number. Otherwise, they start as the same version + # and the latest value is ignored. + td_synchronizer.update(td_initializer.flag("feature-flag").on(True)) + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(False)) + data_system_config = DataSystemConfig( + initializers=[td_initializer.build_initializer], + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + initialized = Event() + modified = Event() + changes: List[FlagChange] = [] + count = 0 + + def listener(flag_change: FlagChange): + nonlocal count, changes + count += 1 + changes.append(flag_change) + + if count == 2: + initialized.set() + if count == 3: + modified.set() + + fdv2.flag_tracker.add_listener(listener) + + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + assert initialized.wait(1), "Flag change listener was not called in time" + + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + assert modified.wait(1), "Flag change listener was not called in time" + assert len(changes) == 3 + assert changes[0].key == "feature-flag" + assert changes[1].key == "feature-flag" + assert changes[2].key == "feature-flag" + + +def test_can_stop_fdv2(): + td = TestDataV2.data_source() + data_system_config = DataSystemConfig( + initializers=None, + primary_synchronizer=td.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + changed = Event() + changes: List[FlagChange] = [] + + def listener(flag_change: FlagChange): + changes.append(flag_change) + changed.set() + + fdv2.flag_tracker.add_listener(listener) + + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + + fdv2.stop() + + td.update(td.flag("feature-flag").on(False)) + assert changed.wait(1) is False, "Flag change listener was erroneously called" + assert len(changes) == 0 + + +def test_fdv2_data_availability_is_refreshed_with_data(): + td = TestDataV2.data_source() + data_system_config = DataSystemConfig( + initializers=None, + primary_synchronizer=td.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + + assert fdv2.data_availability.at_least(DataAvailability.REFRESHED) + assert fdv2.target_availability.at_least(DataAvailability.REFRESHED) + + +def test_fdv2_fallsback_to_secondary_synchronizer(): + mock: Synchronizer = Mock() + mock.sync.return_value = iter([]) # Empty iterator to simulate no data + td = TestDataV2.data_source() + td.update(td.flag("feature-flag").on(True)) + data_system_config = DataSystemConfig( + initializers=[td.build_initializer], + primary_synchronizer=lambda _: mock, # Primary synchronizer is None to force fallback + secondary_synchronizer=td.build_synchronizer, + ) + + changed = Event() + changes: List[FlagChange] = [] + count = 0 + + def listener(flag_change: FlagChange): + nonlocal count, changes + count += 1 + changes.append(flag_change) + + if count == 2: + changed.set() + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + + td.update(td.flag("feature-flag").on(False)) + assert changed.wait(1), "Flag change listener was not called in time" + assert len(changes) == 2 + assert changes[0].key == "feature-flag" + assert changes[1].key == "feature-flag" + + +def test_fdv2_shutdown_down_if_both_synchronizers_fail(): + mock: Synchronizer = Mock() + mock.sync.return_value = iter([]) # Empty iterator to simulate no data + td = TestDataV2.data_source() + td.update(td.flag("feature-flag").on(True)) + data_system_config = DataSystemConfig( + initializers=[td.build_initializer], + primary_synchronizer=lambda _: mock, # Primary synchronizer is None to force fallback + secondary_synchronizer=lambda _: mock, # Secondary synchronizer also fails + ) + + changed = Event() + + def listener(status: DataSourceStatus): + if status.state == DataSourceState.OFF: + changed.set() + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.data_source_status_provider.add_listener(listener) + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + + assert changed.wait(1), "Data system did not shut down in time" + assert fdv2.data_source_status_provider.status.state == DataSourceState.OFF + + +def test_fdv2_falls_back_to_fdv1_on_polling_error_with_header(): + """ + Test that FDv2 falls back to FDv1 when polling receives an error response + with the X-LD-FD-Fallback: true header. + """ + # Create a mock primary synchronizer that signals FDv1 fallback + mock_primary: Synchronizer = Mock() + mock_primary.name = "mock-primary" + mock_primary.stop = Mock() + + # Simulate a synchronizer that yields an OFF state with revert_to_fdv1=True + mock_primary.sync.return_value = iter([ + Update( + state=DataSourceState.OFF, + revert_to_fdv1=True + ) + ]) + + # Create FDv1 fallback data source with actual data + td_fdv1 = TestDataV2.data_source() + td_fdv1.update(td_fdv1.flag("fdv1-flag").on(True)) + + data_system_config = DataSystemConfig( + initializers=None, + primary_synchronizer=lambda _: mock_primary, + fdv1_fallback_synchronizer=td_fdv1.build_synchronizer, + ) + + changed = Event() + changes: List[FlagChange] = [] + + def listener(flag_change: FlagChange): + changes.append(flag_change) + changed.set() + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Update flag in FDv1 data source to verify it's being used + td_fdv1.update(td_fdv1.flag("fdv1-flag").on(False)) + assert changed.wait(1), "Flag change listener was not called in time" + + # Verify we got flag changes from FDv1 + assert len(changes) > 0 + assert any(c.key == "fdv1-flag" for c in changes) + + +def test_fdv2_falls_back_to_fdv1_on_polling_success_with_header(): + """ + Test that FDv2 falls back to FDv1 when polling receives a successful response + with the X-LD-FD-Fallback: true header. + """ + # Create a mock primary synchronizer that yields valid data but signals fallback + mock_primary: Synchronizer = Mock() + mock_primary.name = "mock-primary" + mock_primary.stop = Mock() + + mock_primary.sync.return_value = iter([ + Update( + state=DataSourceState.VALID, + revert_to_fdv1=True + ) + ]) + + # Create FDv1 fallback data source + td_fdv1 = TestDataV2.data_source() + td_fdv1.update(td_fdv1.flag("fdv1-fallback-flag").on(True)) + + data_system_config = DataSystemConfig( + initializers=None, + primary_synchronizer=lambda _: mock_primary, + fdv1_fallback_synchronizer=td_fdv1.build_synchronizer, + ) + + changed = Event() + changes: List[FlagChange] = [] + count = 0 + + def listener(flag_change: FlagChange): + nonlocal count + count += 1 + changes.append(flag_change) + if count >= 2: + changed.set() + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Trigger a flag update in FDv1 + td_fdv1.update(td_fdv1.flag("fdv1-fallback-flag").on(False)) + assert changed.wait(1), "Flag change listener was not called in time" + + # Verify FDv1 is active + assert len(changes) > 0 + assert any(c.key == "fdv1-fallback-flag" for c in changes) + + +def test_fdv2_falls_back_to_fdv1_with_initializer(): + """ + Test that FDv2 falls back to FDv1 even when initialized with data, + and that the FDv1 data replaces the initialized data. + """ + # Initialize with some data + td_initializer = TestDataV2.data_source() + td_initializer.update(td_initializer.flag("initial-flag").on(True)) + + # Create mock primary that signals fallback + mock_primary: Synchronizer = Mock() + mock_primary.name = "mock-primary" + mock_primary.stop = Mock() + + mock_primary.sync.return_value = iter([ + Update( + state=DataSourceState.OFF, + revert_to_fdv1=True + ) + ]) + + # Create FDv1 fallback with different data + td_fdv1 = TestDataV2.data_source() + td_fdv1.update(td_fdv1.flag("fdv1-replacement-flag").on(True)) + + data_system_config = DataSystemConfig( + initializers=[td_initializer.build_initializer], + primary_synchronizer=lambda _: mock_primary, + fdv1_fallback_synchronizer=td_fdv1.build_synchronizer, + ) + + changed = Event() + changes: List[FlagChange] = [] + + def listener(flag_change: FlagChange): + changes.append(flag_change) + if len(changes) >= 2: + changed.set() + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + assert changed.wait(2), "Expected flag changes for both initial and fdv1 flags" + + # Verify we got changes for both flags + flag_keys = [c.key for c in changes] + assert "initial-flag" in flag_keys + assert "fdv1-replacement-flag" in flag_keys + + +def test_fdv2_no_fallback_without_header(): + """ + Test that FDv2 does NOT fall back to FDv1 when an error occurs + but the fallback header is not present. + """ + # Create mock primary that fails but doesn't signal fallback + mock_primary: Synchronizer = Mock() + mock_primary.name = "mock-primary" + mock_primary.stop = Mock() + + mock_primary.sync.return_value = iter([ + Update( + state=DataSourceState.INTERRUPTED, + revert_to_fdv1=False # No fallback + ) + ]) + + # Create mock secondary + mock_secondary: Synchronizer = Mock() + mock_secondary.name = "mock-secondary" + mock_secondary.stop = Mock() + mock_secondary.sync.return_value = iter([ + Update( + state=DataSourceState.VALID, + revert_to_fdv1=False + ) + ]) + + # Create FDv1 fallback (should not be used) + td_fdv1 = TestDataV2.data_source() + td_fdv1.update(td_fdv1.flag("fdv1-should-not-appear").on(True)) + + data_system_config = DataSystemConfig( + initializers=None, + primary_synchronizer=lambda _: mock_primary, + secondary_synchronizer=lambda _: mock_secondary, + fdv1_fallback_synchronizer=td_fdv1.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Verify secondary was called (fallback to secondary, not FDv1) + # Give it a moment to process + import time + time.sleep(0.2) + + # The primary should have been called, then secondary + mock_primary.sync.assert_called() + mock_secondary.sync.assert_called() + + +def test_fdv2_stays_on_fdv1_after_fallback(): + """ + Test that once FDv2 falls back to FDv1, it stays on FDv1 and doesn't + attempt to recover to FDv2. + """ + # Create mock primary that signals fallback + mock_primary: Synchronizer = Mock() + mock_primary.name = "mock-primary" + mock_primary.stop = Mock() + + mock_primary.sync.return_value = iter([ + Update( + state=DataSourceState.OFF, + revert_to_fdv1=True + ) + ]) + + # Create FDv1 fallback + td_fdv1 = TestDataV2.data_source() + td_fdv1.update(td_fdv1.flag("fdv1-flag").on(True)) + + data_system_config = DataSystemConfig( + initializers=None, + primary_synchronizer=lambda _: mock_primary, + fdv1_fallback_synchronizer=td_fdv1.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Give it time to settle + import time + time.sleep(0.5) + + # Primary should only be called once (not retried after fallback) + assert mock_primary.sync.call_count == 1 + + # Verify FDv1 is serving data + store = fdv2.store + flag = store.get(FEATURES, "fdv1-flag", lambda x: x) + assert flag is not None diff --git a/ldclient/testing/impl/datasystem/test_fdv2_persistence.py b/ldclient/testing/impl/datasystem/test_fdv2_persistence.py new file mode 100644 index 00000000..a59fc772 --- /dev/null +++ b/ldclient/testing/impl/datasystem/test_fdv2_persistence.py @@ -0,0 +1,781 @@ +# pylint: disable=missing-docstring + +from threading import Event +from typing import Any, Callable, Dict, List, Mapping, Optional + +from ldclient.config import Config, DataSystemConfig +from ldclient.impl.datasystem import DataAvailability +from ldclient.impl.datasystem.fdv2 import FDv2 +from ldclient.integrations.test_datav2 import TestDataV2 +from ldclient.interfaces import DataStoreMode, FeatureStore, FlagChange +from ldclient.versioned_data_kind import FEATURES, SEGMENTS, VersionedDataKind + + +class StubFeatureStore(FeatureStore): + """ + A simple stub implementation of FeatureStore for testing. + Records all operations and allows inspection of state. + """ + + def __init__( + self, + initial_data: Optional[ + Dict[VersionedDataKind, Dict[str, Dict[Any, Any]]] + ] = None, + ): + self._data: Dict[VersionedDataKind, Dict[str, dict]] = { + FEATURES: {}, + SEGMENTS: {}, + } + self._initialized = False + self._available = True + self._monitoring_enabled = False + + # Track operations for assertions + self.init_called_count = 0 + self.upsert_calls: List[tuple] = [] + self.delete_calls: List[tuple] = [] + self.get_calls: List[tuple] = [] + self.all_calls: List[VersionedDataKind] = [] + + if initial_data: + self.init(initial_data) + + def init(self, all_data: Mapping[VersionedDataKind, Mapping[str, Dict[Any, Any]]]): + self.init_called_count += 1 + self._data = { + FEATURES: dict(all_data.get(FEATURES, {})), + SEGMENTS: dict(all_data.get(SEGMENTS, {})), + } + self._initialized = True + + def get( + self, + kind: VersionedDataKind, + key: str, + callback: Callable[[Any], Any] = lambda x: x, + ): + self.get_calls.append((kind, key)) + item = self._data.get(kind, {}).get(key) + return callback(item) if item else None + + def all( + self, kind: VersionedDataKind, callback: Callable[[Any], Any] = lambda x: x + ): + self.all_calls.append(kind) + items = self._data.get(kind, {}) + return {key: callback(value) for key, value in items.items()} + + def delete(self, kind: VersionedDataKind, key: str, version: int): + self.delete_calls.append((kind, key, version)) + existing = self._data.get(kind, {}).get(key) + if existing and existing.get("version", 0) < version: + self._data[kind][key] = {"key": key, "version": version, "deleted": True} + + def upsert(self, kind: VersionedDataKind, item: dict): + self.upsert_calls.append((kind, item.get("key"), item.get("version"))) + key = item["key"] + existing = self._data.get(kind, {}).get(key) + if not existing or existing.get("version", 0) < item.get("version", 0): + self._data[kind][key] = item + + @property + def initialized(self) -> bool: + return self._initialized + + def is_available(self) -> bool: + """For monitoring support""" + return self._available + + def is_monitoring_enabled(self) -> bool: + """For monitoring support""" + return self._monitoring_enabled + + def set_available(self, available: bool): + """Test helper to simulate availability changes""" + self._available = available + + def enable_monitoring(self): + """Test helper to enable monitoring""" + self._monitoring_enabled = True + + def get_data_snapshot(self) -> Mapping[VersionedDataKind, Mapping[str, dict]]: + """Test helper to get a snapshot of current data""" + return { + FEATURES: dict(self._data[FEATURES]), + SEGMENTS: dict(self._data[SEGMENTS]), + } + + def reset_operation_tracking(self): + """Test helper to reset operation tracking""" + self.init_called_count = 0 + self.upsert_calls = [] + self.delete_calls = [] + self.get_calls = [] + self.all_calls = [] + + +def test_persistent_store_read_only_mode(): + """Test that READ_ONLY mode reads from store but never writes""" + # Pre-populate persistent store with a flag + initial_data = { + FEATURES: { + "existing-flag": { + "key": "existing-flag", + "version": 1, + "on": True, + "variations": [True, False], + "fallthrough": {"variation": 0}, + } + }, + SEGMENTS: {}, + } + + persistent_store = StubFeatureStore(initial_data) + + # Create synchronizer that will provide new data + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("new-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_ONLY, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Verify data system is initialized and available + assert fdv2.data_availability.at_least(DataAvailability.REFRESHED) + + # Verify the store was initialized once (by us) but no additional writes happened + # The persistent store should have been read from, but not written to + assert persistent_store.init_called_count == 1 # Only our initial setup + assert len(persistent_store.upsert_calls) == 0 # No upserts in READ_ONLY mode + + fdv2.stop() + + +def test_persistent_store_read_write_mode(): + """Test that READ_WRITE mode reads from store and writes updates back""" + # Pre-populate persistent store with a flag + initial_data = { + FEATURES: { + "existing-flag": { + "key": "existing-flag", + "version": 1, + "on": True, + "variations": [True, False], + "fallthrough": {"variation": 0}, + } + }, + SEGMENTS: {}, + } + + persistent_store = StubFeatureStore(initial_data) + persistent_store.reset_operation_tracking() # Reset tracking after initial setup + + # Create synchronizer that will provide new data + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("new-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # In READ_WRITE mode, the store should be initialized with new data + assert ( + persistent_store.init_called_count >= 1 + ) # At least one init call for the new data + + # Verify the new flag was written to persistent store + snapshot = persistent_store.get_data_snapshot() + assert "new-flag" in snapshot[FEATURES] + + fdv2.stop() + + +def test_persistent_store_delta_updates_read_write(): + """Test that delta updates are written to persistent store in READ_WRITE mode""" + persistent_store = StubFeatureStore() + + # Create synchronizer + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Set up flag change listener to detect the update + flag_changed = Event() + change_count = 0 + + def listener(flag_change: FlagChange): + nonlocal change_count + change_count += 1 + if ( + change_count == 2 + ): # First change is from initial sync, second is our update + flag_changed.set() + + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + persistent_store.reset_operation_tracking() + + # Make a delta update + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(False)) + + # Wait for the flag change to propagate + assert flag_changed.wait(1), "Flag change did not propagate in time" + + # Verify the update was written to persistent store + assert len(persistent_store.upsert_calls) > 0 + assert any(call[1] == "feature-flag" for call in persistent_store.upsert_calls) + + # Verify the updated flag is in the store + snapshot = persistent_store.get_data_snapshot() + assert "feature-flag" in snapshot[FEATURES] + assert snapshot[FEATURES]["feature-flag"]["on"] is False + + fdv2.stop() + + +def test_persistent_store_delta_updates_read_only(): + """Test that delta updates are NOT written to persistent store in READ_ONLY mode""" + persistent_store = StubFeatureStore() + + # Create synchronizer + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_ONLY, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Set up flag change listener to detect the update + flag_changed = Event() + change_count = [0] # Use list to allow modification in nested function + + def listener(flag_change: FlagChange): + change_count[0] += 1 + if ( + change_count[0] == 2 + ): # First change is from initial sync, second is our update + flag_changed.set() + + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + persistent_store.reset_operation_tracking() + + # Make a delta update + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(False)) + + # Wait for the flag change to propagate + assert flag_changed.wait(1), "Flag change did not propagate in time" + + # Verify NO updates were written to persistent store in READ_ONLY mode + assert len(persistent_store.upsert_calls) == 0 + + fdv2.stop() + + +def test_persistent_store_with_initializer_and_synchronizer(): + """Test that both initializer and synchronizer data are persisted in READ_WRITE mode""" + persistent_store = StubFeatureStore() + + # Create initializer with one flag + td_initializer = TestDataV2.data_source() + td_initializer.update(td_initializer.flag("init-flag").on(True)) + + # Create synchronizer with another flag + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("sync-flag").on(False)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=[td_initializer.build_initializer], + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Set up flag change listener to detect when synchronizer data arrives + sync_flag_arrived = Event() + + def listener(flag_change: FlagChange): + if flag_change.key == "sync-flag": + sync_flag_arrived.set() + + fdv2.flag_tracker.add_listener(listener) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Wait for synchronizer to fully initialize + # The synchronizer does a full data set transfer, so it replaces the initializer data + assert sync_flag_arrived.wait(1), "Synchronizer data did not arrive in time" + + # The synchronizer flag should be in the persistent store + # (it replaces the init-flag since synchronizer does a full data set) + snapshot = persistent_store.get_data_snapshot() + assert "init-flag" not in snapshot[FEATURES] + assert "sync-flag" in snapshot[FEATURES] + + fdv2.stop() + + +def test_persistent_store_delete_operations(): + """Test that delete operations are written to persistent store in READ_WRITE mode""" + # We'll need to manually trigger a delete via the store + # This is more of an integration test with the Store class + from ldclient.impl.datasystem.store import Store + from ldclient.impl.listeners import Listeners + from ldclient.interfaces import ( + Change, + ChangeSet, + ChangeType, + IntentCode, + ObjectKind + ) + + # Pre-populate with a flag + initial_data = { + FEATURES: { + "deletable-flag": { + "key": "deletable-flag", + "version": 1, + "on": True, + "variations": [True, False], + "fallthrough": {"variation": 0}, + } + }, + SEGMENTS: {}, + } + + persistent_store = StubFeatureStore(initial_data) + + store = Store(Listeners(), Listeners()) + store.with_persistence(persistent_store, True, None) + + # First, initialize the store with the data so it's in memory + init_changeset = ChangeSet( + intent_code=IntentCode.TRANSFER_FULL, + changes=[ + Change( + action=ChangeType.PUT, + kind=ObjectKind.FLAG, + key="deletable-flag", + version=1, + object={ + "key": "deletable-flag", + "version": 1, + "on": True, + "variations": [True, False], + "fallthrough": {"variation": 0}, + }, + ) + ], + selector=None, + ) + store.apply(init_changeset, True) + + persistent_store.reset_operation_tracking() + + # Now apply a changeset with a delete + delete_changeset = ChangeSet( + intent_code=IntentCode.TRANSFER_CHANGES, + changes=[ + Change( + action=ChangeType.DELETE, + kind=ObjectKind.FLAG, + key="deletable-flag", + version=2, + object=None, + ) + ], + selector=None, + ) + + store.apply(delete_changeset, True) + + # Verify delete was called on persistent store + assert len(persistent_store.upsert_calls) > 0 + assert any(call[1] == "deletable-flag" for call in persistent_store.upsert_calls) + + +def test_data_store_status_provider(): + """Test that data store status provider is correctly initialized""" + persistent_store = StubFeatureStore() + + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Verify data store status provider exists + status_provider = fdv2.data_store_status_provider + assert status_provider is not None + + # Get initial status + status = status_provider.status + assert status is not None + assert status.available is True + + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + + fdv2.stop() + + +def test_data_store_status_monitoring_not_enabled_by_default(): + """Test that monitoring is not enabled by default""" + persistent_store = StubFeatureStore() + + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Monitoring should not be enabled because the store doesn't support it + status_provider = fdv2.data_store_status_provider + assert status_provider.is_monitoring_enabled() is False + + +def test_data_store_status_monitoring_enabled_when_supported(): + """Test that monitoring is enabled when the store supports it""" + persistent_store = StubFeatureStore() + persistent_store.enable_monitoring() + + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Monitoring should be enabled + status_provider = fdv2.data_store_status_provider + assert status_provider.is_monitoring_enabled() is True + + +def test_no_persistent_store_status_provider_without_store(): + """Test that data store status provider exists even without a persistent store""" + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=None, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + + # Status provider should exist but not be monitoring + status_provider = fdv2.data_store_status_provider + assert status_provider is not None + assert status_provider.is_monitoring_enabled() is False + + fdv2.start(set_on_ready) + assert set_on_ready.wait(1), "Data system did not become ready in time" + + fdv2.stop() + + +def test_persistent_store_outage_recovery_flushes_on_recovery(): + """Test that in-memory store is flushed to persistent store when it recovers from outage""" + from ldclient.interfaces import DataStoreStatus + + persistent_store = StubFeatureStore() + + # Create synchronizer with initial data + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Verify initial data is in the persistent store + snapshot = persistent_store.get_data_snapshot() + assert "feature-flag" in snapshot[FEATURES] + assert snapshot[FEATURES]["feature-flag"]["on"] is True + + # Reset tracking to isolate recovery behavior + persistent_store.reset_operation_tracking() + + event = Event() + fdv2.flag_tracker.add_listener(lambda _flag_change: event.set()) + # Simulate a new flag being added while store is "offline" + # (In reality, the store is still online, but we're testing the recovery mechanism) + td_synchronizer.update(td_synchronizer.flag("new-flag").on(False)) + + # Block until the flag has propagated through the data store + assert event.wait(1) + + # Now simulate the persistent store coming back online with stale data + # by triggering the recovery callback directly + fdv2._persistent_store_outage_recovery(DataStoreStatus(available=True, stale=True)) + + # Verify that init was called on the persistent store (flushing in-memory data) + assert persistent_store.init_called_count > 0, "Store should have been reinitialized" + + # Verify both flags are now in the persistent store + snapshot = persistent_store.get_data_snapshot() + assert "feature-flag" in snapshot[FEATURES] + assert "new-flag" in snapshot[FEATURES] + + fdv2.stop() + + +def test_persistent_store_outage_recovery_no_flush_when_not_stale(): + """Test that recovery does NOT flush when store comes back online without stale data""" + from ldclient.interfaces import DataStoreStatus + + persistent_store = StubFeatureStore() + + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Reset tracking + persistent_store.reset_operation_tracking() + + # Simulate store coming back online but NOT stale (data is fresh) + fdv2._persistent_store_outage_recovery(DataStoreStatus(available=True, stale=False)) + + # Verify that init was NOT called (no flush needed) + assert persistent_store.init_called_count == 0, "Store should not be reinitialized when not stale" + + fdv2.stop() + + +def test_persistent_store_outage_recovery_no_flush_when_unavailable(): + """Test that recovery does NOT flush when store is unavailable""" + from ldclient.interfaces import DataStoreStatus + + persistent_store = StubFeatureStore() + + td_synchronizer = TestDataV2.data_source() + td_synchronizer.update(td_synchronizer.flag("feature-flag").on(True)) + + data_system_config = DataSystemConfig( + data_store_mode=DataStoreMode.READ_WRITE, + data_store=persistent_store, + initializers=None, + primary_synchronizer=td_synchronizer.build_synchronizer, + ) + + set_on_ready = Event() + fdv2 = FDv2(Config(sdk_key="dummy"), data_system_config) + fdv2.start(set_on_ready) + + assert set_on_ready.wait(1), "Data system did not become ready in time" + + # Reset tracking + persistent_store.reset_operation_tracking() + + # Simulate store being unavailable (even if marked as stale) + fdv2._persistent_store_outage_recovery(DataStoreStatus(available=False, stale=True)) + + # Verify that init was NOT called (store is not available) + assert persistent_store.init_called_count == 0, "Store should not be reinitialized when unavailable" + + fdv2.stop() + + +def test_persistent_store_commit_encodes_data_correctly(): + """Test that Store.commit() properly encodes data before writing to persistent store""" + from ldclient.impl.datasystem.store import Store + from ldclient.impl.listeners import Listeners + from ldclient.interfaces import ( + Change, + ChangeSet, + ChangeType, + IntentCode, + ObjectKind + ) + + persistent_store = StubFeatureStore() + store = Store(Listeners(), Listeners()) + store.with_persistence(persistent_store, True, None) + + # Create a flag with raw data + flag_data = { + "key": "test-flag", + "version": 1, + "on": True, + "variations": [True, False], + "fallthrough": {"variation": 0}, + } + + # Apply a changeset to add the flag to the in-memory store + changeset = ChangeSet( + intent_code=IntentCode.TRANSFER_FULL, + changes=[ + Change( + action=ChangeType.PUT, + kind=ObjectKind.FLAG, + key="test-flag", + version=1, + object=flag_data, + ) + ], + selector=None, + ) + store.apply(changeset, True) + + # Reset tracking + persistent_store.reset_operation_tracking() + + # Now commit the in-memory store to the persistent store + err = store.commit() + assert err is None, "Commit should succeed" + + # Verify that init was called with properly encoded data + assert persistent_store.init_called_count == 1, "Init should be called once" + + # Verify the data in the persistent store is properly encoded + snapshot = persistent_store.get_data_snapshot() + assert "test-flag" in snapshot[FEATURES] + + # The data should be in the encoded format (as a dict with all required fields) + flag_in_store = snapshot[FEATURES]["test-flag"] + assert flag_in_store["key"] == "test-flag" + assert flag_in_store["version"] == 1 + assert flag_in_store["on"] is True + + +def test_persistent_store_commit_with_no_persistent_store(): + """Test that Store.commit() safely handles the case where there's no persistent store""" + from ldclient.impl.datasystem.store import Store + from ldclient.impl.listeners import Listeners + + # Create store without persistent store + store = Store(Listeners(), Listeners()) + + # Commit should succeed but do nothing + err = store.commit() + assert err is None, "Commit should succeed even without persistent store" + + +def test_persistent_store_commit_handles_errors(): + """Test that Store.commit() handles errors from persistent store gracefully""" + from ldclient.impl.datasystem.store import Store + from ldclient.impl.listeners import Listeners + from ldclient.interfaces import ( + Change, + ChangeSet, + ChangeType, + IntentCode, + ObjectKind + ) + + class FailingFeatureStore(StubFeatureStore): + """A feature store that always fails on init""" + def init(self, all_data): + raise RuntimeError("Simulated persistent store failure") + + persistent_store = FailingFeatureStore() + store = Store(Listeners(), Listeners()) + store.with_persistence(persistent_store, True, None) + + # Add some data to the in-memory store + changeset = ChangeSet( + intent_code=IntentCode.TRANSFER_FULL, + changes=[ + Change( + action=ChangeType.PUT, + kind=ObjectKind.FLAG, + key="test-flag", + version=1, + object={"key": "test-flag", "version": 1, "on": True}, + ) + ], + selector=None, + ) + store.apply(changeset, True) + + # Commit should return the error without raising + err = store.commit() + assert err is not None, "Commit should return error from persistent store" + assert isinstance(err, RuntimeError) + assert str(err) == "Simulated persistent store failure" diff --git a/ldclient/testing/integrations/test_file_data_sourcev2.py b/ldclient/testing/integrations/test_file_data_sourcev2.py new file mode 100644 index 00000000..c588ad47 --- /dev/null +++ b/ldclient/testing/integrations/test_file_data_sourcev2.py @@ -0,0 +1,475 @@ +import json +import os +import tempfile +import threading +import time + +import pytest + +from ldclient.config import Config +from ldclient.impl.util import _Fail, _Success +from ldclient.integrations import Files +from ldclient.interfaces import ( + DataSourceState, + IntentCode, + ObjectKind, + Selector +) +from ldclient.testing.mock_components import MockSelectorStore + +# Skip all tests in this module in CI due to flakiness +pytestmark = pytest.mark.skipif( + os.getenv('LD_SKIP_FLAKY_TESTS', '').lower() in ('true', '1', 'yes'), + reason="Skipping flaky test" +) + +have_yaml = False +try: + import yaml + have_yaml = True +except ImportError: + pass + + +all_properties_json = ''' +{ + "flags": { + "flag1": { + "key": "flag1", + "on": true, + "fallthrough": { + "variation": 2 + }, + "variations": [ "fall", "off", "on" ] + } + }, + "flagValues": { + "flag2": "value2" + }, + "segments": { + "seg1": { + "key": "seg1", + "include": ["user1"] + } + } +} +''' + +all_properties_yaml = ''' +--- +flags: + flag1: + key: flag1 + "on": true +flagValues: + flag2: value2 +segments: + seg1: + key: seg1 + include: ["user1"] +''' + +flag_only_json = ''' +{ + "flags": { + "flag1": { + "key": "flag1", + "on": true, + "fallthrough": { + "variation": 2 + }, + "variations": [ "fall", "off", "on" ] + } + } +} +''' + +segment_only_json = ''' +{ + "segments": { + "seg1": { + "key": "seg1", + "include": ["user1"] + } + } +} +''' + +flag_values_only_json = ''' +{ + "flagValues": { + "flag2": "value2" + } +} +''' + + +def make_temp_file(content): + """Create a temporary file with the given content.""" + f, path = tempfile.mkstemp() + os.write(f, content.encode("utf-8")) + os.close(f) + return path + + +def replace_file(path, content): + """Replace the contents of a file.""" + with open(path, 'w') as f: + f.write(content) + + +def test_creates_valid_initializer(): + """Test that FileDataSourceV2 creates a working initializer.""" + path = make_temp_file(all_properties_json) + try: + file_source = Files.new_data_source_v2(paths=[path]) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Success) + + basis = result.value + assert not basis.persist + assert basis.environment_id is None + assert basis.change_set.intent_code == IntentCode.TRANSFER_FULL + + # Should have 2 flags and 1 segment + changes = basis.change_set.changes + assert len(changes) == 3 + + flag_changes = [c for c in changes if c.kind == ObjectKind.FLAG] + segment_changes = [c for c in changes if c.kind == ObjectKind.SEGMENT] + + assert len(flag_changes) == 2 + assert len(segment_changes) == 1 + + # Check selector is no_selector + assert basis.change_set.selector == Selector.no_selector() + finally: + os.remove(path) + + +def test_initializer_handles_missing_file(): + """Test that initializer returns error for missing file.""" + file_source = Files.new_data_source_v2(paths=['no-such-file.json']) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Fail) + assert "no-such-file.json" in result.error + + +def test_initializer_handles_invalid_json(): + """Test that initializer returns error for invalid JSON.""" + path = make_temp_file('{"flagValues":{') + try: + file_source = Files.new_data_source_v2(paths=[path]) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Fail) + assert "Unable to load flag data" in result.error + finally: + os.remove(path) + + +def test_initializer_handles_duplicate_keys(): + """Test that initializer returns error when same key appears in multiple files.""" + path1 = make_temp_file(flag_only_json) + path2 = make_temp_file(flag_only_json) + try: + file_source = Files.new_data_source_v2(paths=[path1, path2]) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Fail) + assert "was used more than once" in result.error + finally: + os.remove(path1) + os.remove(path2) + + +def test_initializer_loads_multiple_files(): + """Test that initializer can load from multiple files.""" + path1 = make_temp_file(flag_only_json) + path2 = make_temp_file(segment_only_json) + try: + file_source = Files.new_data_source_v2(paths=[path1, path2]) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Success) + + changes = result.value.change_set.changes + flag_changes = [c for c in changes if c.kind == ObjectKind.FLAG] + segment_changes = [c for c in changes if c.kind == ObjectKind.SEGMENT] + + assert len(flag_changes) == 1 + assert len(segment_changes) == 1 + finally: + os.remove(path1) + os.remove(path2) + + +def test_initializer_loads_yaml(): + """Test that initializer can parse YAML files.""" + if not have_yaml: + pytest.skip("skipping YAML test because pyyaml isn't available") + + path = make_temp_file(all_properties_yaml) + try: + file_source = Files.new_data_source_v2(paths=[path]) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Success) + + changes = result.value.change_set.changes + assert len(changes) == 3 # 2 flags + 1 segment + finally: + os.remove(path) + + +def test_initializer_handles_flag_values(): + """Test that initializer properly converts flagValues to flags.""" + path = make_temp_file(flag_values_only_json) + try: + file_source = Files.new_data_source_v2(paths=[path]) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Success) + + changes = result.value.change_set.changes + flag_changes = [c for c in changes if c.kind == ObjectKind.FLAG] + assert len(flag_changes) == 1 + + # Check the flag was created with the expected structure + flag_change = flag_changes[0] + assert flag_change.key == "flag2" + assert flag_change.object['key'] == "flag2" + assert flag_change.object['on'] is True + assert flag_change.object['variations'] == ["value2"] + finally: + os.remove(path) + + +def test_creates_valid_synchronizer(): + """Test that FileDataSourceV2 creates a working synchronizer.""" + path = make_temp_file(all_properties_json) + try: + file_source = Files.new_data_source_v2(paths=[path], force_polling=True, poll_interval=0.1) + synchronizer = file_source(Config(sdk_key="dummy")) + + updates = [] + update_count = 0 + + def collect_updates(): + nonlocal update_count + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): + updates.append(update) + update_count += 1 + + if update_count == 1: + # Should get initial state + assert update.state == DataSourceState.VALID + assert update.change_set is not None + assert update.change_set.intent_code == IntentCode.TRANSFER_FULL + assert len(update.change_set.changes) == 3 + synchronizer.stop() + break + + # Start the synchronizer in a thread with timeout to prevent hanging + sync_thread = threading.Thread(target=collect_updates) + sync_thread.start() + + # Wait for the thread to complete with timeout + sync_thread.join(timeout=5) + + # Ensure thread completed successfully + if sync_thread.is_alive(): + synchronizer.stop() + sync_thread.join() + pytest.fail("Synchronizer test timed out after 5 seconds") + + assert len(updates) == 1 + finally: + synchronizer.stop() + os.remove(path) + + +def test_synchronizer_detects_file_changes(): + """Test that synchronizer detects and reports file changes.""" + path = make_temp_file(flag_only_json) + try: + file_source = Files.new_data_source_v2(paths=[path], force_polling=True, poll_interval=0.1) + synchronizer = file_source(Config(sdk_key="dummy")) + + updates = [] + update_event = threading.Event() + + def collect_updates(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): + updates.append(update) + update_event.set() + + if len(updates) >= 2: + break + + # Start the synchronizer + sync_thread = threading.Thread(target=collect_updates) + sync_thread.start() + + # Wait for initial update + assert update_event.wait(timeout=2), "Did not receive initial update" + assert len(updates) == 1 + assert updates[0].state == DataSourceState.VALID + initial_changes = [c for c in updates[0].change_set.changes if c.kind == ObjectKind.FLAG] + assert len(initial_changes) == 1 + + # Modify the file + update_event.clear() + time.sleep(0.2) # Ensure filesystem timestamp changes + replace_file(path, segment_only_json) + + # Wait for the change to be detected + assert update_event.wait(timeout=2), "Did not receive update after file change" + assert len(updates) == 2 + assert updates[1].state == DataSourceState.VALID + segment_changes = [c for c in updates[1].change_set.changes if c.kind == ObjectKind.SEGMENT] + assert len(segment_changes) == 1 + + synchronizer.stop() + sync_thread.join(timeout=2) + finally: + synchronizer.stop() + os.remove(path) + + +def test_synchronizer_reports_error_on_invalid_file_update(): + """Test that synchronizer reports error when file becomes invalid.""" + path = make_temp_file(flag_only_json) + try: + file_source = Files.new_data_source_v2(paths=[path], force_polling=True, poll_interval=0.1) + synchronizer = file_source(Config(sdk_key="dummy")) + + updates = [] + update_event = threading.Event() + + def collect_updates(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): + updates.append(update) + update_event.set() + + if len(updates) >= 2: + break + + # Start the synchronizer + sync_thread = threading.Thread(target=collect_updates) + sync_thread.start() + + # Wait for initial update + assert update_event.wait(timeout=2), "Did not receive initial update" + assert len(updates) == 1 + assert updates[0].state == DataSourceState.VALID + + # Make the file invalid + update_event.clear() + time.sleep(0.2) # Ensure filesystem timestamp changes + replace_file(path, '{"invalid json') + + # Wait for the error to be detected + assert update_event.wait(timeout=2), "Did not receive update after file became invalid" + assert len(updates) == 2 + assert updates[1].state == DataSourceState.INTERRUPTED + assert updates[1].error is not None + + synchronizer.stop() + sync_thread.join(timeout=2) + finally: + synchronizer.stop() + os.remove(path) + + +def test_synchronizer_can_be_stopped(): + """Test that synchronizer stops cleanly.""" + path = make_temp_file(all_properties_json) + try: + file_source = Files.new_data_source_v2(paths=[path]) + synchronizer = file_source(Config(sdk_key="dummy")) + + updates = [] + + def collect_updates(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): + updates.append(update) + + # Start the synchronizer + sync_thread = threading.Thread(target=collect_updates) + sync_thread.start() + + # Give it a moment to process initial data + time.sleep(0.2) + + # Stop it + synchronizer.stop() + + # Thread should complete + sync_thread.join(timeout=2) + assert not sync_thread.is_alive() + + # Should have received at least the initial update + assert len(updates) >= 1 + assert updates[0].state == DataSourceState.VALID + finally: + os.remove(path) + + +def test_fetch_after_stop_returns_error(): + """Test that fetch returns error after synchronizer is stopped.""" + path = make_temp_file(all_properties_json) + try: + file_source = Files.new_data_source_v2(paths=[path]) + initializer = file_source(Config(sdk_key="dummy")) + + # First fetch should work + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Success) + + # Stop the source + initializer.stop() + + # Second fetch should fail + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Fail) + assert "closed" in result.error + finally: + os.remove(path) + + +def test_source_name_property(): + """Test that the data source has the correct name.""" + path = make_temp_file(all_properties_json) + try: + file_source = Files.new_data_source_v2(paths=[path]) + source = file_source(Config(sdk_key="dummy")) + + assert source.name == "FileDataV2" + finally: + source.stop() + os.remove(path) + + +def test_accepts_single_path_string(): + """Test that paths parameter can be a single string.""" + path = make_temp_file(flag_only_json) + try: + # Pass a single string instead of a list + file_source = Files.new_data_source_v2(paths=path) + initializer = file_source(Config(sdk_key="dummy")) + + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) + assert isinstance(result, _Success) + assert len(result.value.change_set.changes) == 1 + finally: + os.remove(path) diff --git a/ldclient/testing/integrations/test_test_data_sourcev2.py b/ldclient/testing/integrations/test_test_data_sourcev2.py index ac52278a..177a6af5 100644 --- a/ldclient/testing/integrations/test_test_data_sourcev2.py +++ b/ldclient/testing/integrations/test_test_data_sourcev2.py @@ -4,14 +4,17 @@ import pytest -from ldclient.impl.datasystem.protocolv2 import ( +from ldclient.config import Config +from ldclient.impl.util import _Fail, _Success +from ldclient.integrations.test_datav2 import FlagBuilderV2, TestDataV2 +from ldclient.interfaces import ( ChangeType, + DataSourceState, IntentCode, - ObjectKind + ObjectKind, + Selector ) -from ldclient.impl.util import _Fail, _Success -from ldclient.integrations.test_datav2 import FlagBuilderV2, TestDataV2 -from ldclient.interfaces import DataSourceState +from ldclient.testing.mock_components import MockSelectorStore # Test Data + Data Source V2 @@ -19,9 +22,9 @@ def test_creates_valid_initializer(): """Test that TestDataV2 creates a working initializer""" td = TestDataV2.data_source() - initializer = td.build_initializer() + initializer = td.build_initializer(Config(sdk_key="dummy")) - result = initializer.fetch() + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Success) basis = result.value @@ -34,14 +37,14 @@ def test_creates_valid_initializer(): def test_creates_valid_synchronizer(): """Test that TestDataV2 creates a working synchronizer""" td = TestDataV2.data_source() - synchronizer = td.build_synchronizer() + synchronizer = td.build_synchronizer(Config(sdk_key="dummy")) updates = [] update_count = 0 def collect_updates(): nonlocal update_count - for update in synchronizer.sync(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): updates.append(update) update_count += 1 @@ -50,7 +53,7 @@ def collect_updates(): assert update.state == DataSourceState.VALID assert update.change_set is not None assert update.change_set.intent_code == IntentCode.TRANSFER_FULL - synchronizer.close() + synchronizer.stop() break # Start the synchronizer in a thread with timeout to prevent hanging @@ -62,7 +65,7 @@ def collect_updates(): # Ensure thread completed successfully if sync_thread.is_alive(): - synchronizer.close() + synchronizer.stop() sync_thread.join() pytest.fail("Synchronizer test timed out after 5 seconds") @@ -238,8 +241,8 @@ def test_initializer_fetches_flag_data(): td = TestDataV2.data_source() td.update(td.flag('some-flag').variation_for_all(True)) - initializer = td.build_initializer() - result = initializer.fetch() + initializer = td.build_initializer(Config(sdk_key="dummy")) + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Success) basis = result.value @@ -258,9 +261,9 @@ def test_synchronizer_yields_initial_data(): td = TestDataV2.data_source() td.update(td.flag('initial-flag').variation_for_all(False)) - synchronizer = td.build_synchronizer() + synchronizer = td.build_synchronizer(Config(sdk_key="dummy")) - update_iter = iter(synchronizer.sync()) + update_iter = iter(synchronizer.sync(MockSelectorStore(Selector.no_selector()))) initial_update = next(update_iter) assert initial_update.state == DataSourceState.VALID @@ -271,25 +274,25 @@ def test_synchronizer_yields_initial_data(): change = initial_update.change_set.changes[0] assert change.key == 'initial-flag' - synchronizer.close() + synchronizer.stop() def test_synchronizer_receives_updates(): """Test that synchronizer receives flag updates""" td = TestDataV2.data_source() - synchronizer = td.build_synchronizer() + synchronizer = td.build_synchronizer(Config(sdk_key="dummy")) updates = [] update_count = 0 def collect_updates(): nonlocal update_count - for update in synchronizer.sync(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): updates.append(update) update_count += 1 if update_count >= 2: - synchronizer.close() + synchronizer.stop() break # Start the synchronizer in a thread @@ -321,24 +324,24 @@ def collect_updates(): def test_multiple_synchronizers_receive_updates(): """Test that multiple synchronizers receive the same updates""" td = TestDataV2.data_source() - sync1 = td.build_synchronizer() - sync2 = td.build_synchronizer() + sync1 = td.build_synchronizer(Config(sdk_key="dummy")) + sync2 = td.build_synchronizer(Config(sdk_key="dummy")) updates1 = [] updates2 = [] def collect_updates_1(): - for update in sync1.sync(): + for update in sync1.sync(MockSelectorStore(Selector.no_selector())): updates1.append(update) if len(updates1) >= 2: - sync1.close() + sync1.stop() break def collect_updates_2(): - for update in sync2.sync(): + for update in sync2.sync(MockSelectorStore(Selector.no_selector())): updates2.append(update) if len(updates2) >= 2: - sync2.close() + sync2.stop() break # Start both synchronizers @@ -367,14 +370,14 @@ def collect_updates_2(): def test_closed_synchronizer_stops_yielding(): """Test that closed synchronizer stops yielding updates""" td = TestDataV2.data_source() - synchronizer = td.build_synchronizer() + synchronizer = td.build_synchronizer(Config(sdk_key="dummy")) updates = [] # Get initial update then close - for update in synchronizer.sync(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): updates.append(update) - synchronizer.close() + synchronizer.stop() break assert len(updates) == 1 @@ -384,7 +387,7 @@ def test_closed_synchronizer_stops_yielding(): # Try to get more updates - should get an error state indicating closure additional_updates = [] - for update in synchronizer.sync(): + for update in synchronizer.sync(MockSelectorStore(Selector.no_selector())): additional_updates.append(update) break @@ -399,12 +402,13 @@ def test_initializer_can_sync(): td = TestDataV2.data_source() td.update(td.flag('test-flag').variation_for_all(True)) - initializer = td.build_initializer() - sync_gen = initializer.sync() + initializer = td.build_initializer(Config(sdk_key="dummy")) + sync_gen = initializer.sync(MockSelectorStore(Selector.no_selector())) # Should get initial update with data initial_update = next(sync_gen) assert initial_update.state == DataSourceState.VALID + assert initial_update.change_set is not None assert initial_update.change_set.intent_code == IntentCode.TRANSFER_FULL assert len(initial_update.change_set.changes) == 1 assert initial_update.change_set.changes[0].key == 'test-flag' @@ -438,11 +442,11 @@ def test_version_increment(): def test_error_handling_in_fetch(): """Test error handling in the fetch method""" td = TestDataV2.data_source() - initializer = td.build_initializer() + initializer = td.build_initializer(Config(sdk_key="dummy")) # Close the initializer to trigger error condition - initializer.close() + initializer.stop() - result = initializer.fetch() + result = initializer.fetch(MockSelectorStore(Selector.no_selector())) assert isinstance(result, _Fail) assert "TestDataV2 source has been closed" in result.error diff --git a/ldclient/testing/mock_components.py b/ldclient/testing/mock_components.py index 44d3f78a..ad93b32b 100644 --- a/ldclient/testing/mock_components.py +++ b/ldclient/testing/mock_components.py @@ -1,7 +1,11 @@ import time from typing import Callable -from ldclient.interfaces import BigSegmentStore, BigSegmentStoreMetadata +from ldclient.interfaces import ( + BigSegmentStore, + BigSegmentStoreMetadata, + Selector +) class MockBigSegmentStore(BigSegmentStore): @@ -42,3 +46,11 @@ def membership_queries(self) -> list: def __fail(self): raise Exception("deliberate error") + + +class MockSelectorStore(): + def __init__(self, selector: Selector): + self._selector = selector + + def selector(self) -> Selector: + return self._selector diff --git a/ldclient/testing/test_file_data_source.py b/ldclient/testing/test_file_data_source.py index 62646d9e..b8e3fb0b 100644 --- a/ldclient/testing/test_file_data_source.py +++ b/ldclient/testing/test_file_data_source.py @@ -21,6 +21,12 @@ from ldclient.testing.test_util import SpyListener from ldclient.versioned_data_kind import FEATURES, SEGMENTS +# Skip all tests in this module in CI due to flakiness +pytestmark = pytest.mark.skipif( + os.getenv('LD_SKIP_FLAKY_TESTS', '').lower() in ('true', '1', 'yes'), + reason="Skipping flaky test" +) + have_yaml = False try: import yaml diff --git a/pyproject.toml b/pyproject.toml index 93664d02..7871a387 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -33,7 +33,7 @@ expiringdict = ">=1.1.4" pyRFC3339 = ">=1.0" semver = ">=2.10.2" urllib3 = ">=1.26.0,<3" -launchdarkly-eventsource = ">=1.2.4,<2.0.0" +launchdarkly-eventsource = ">=1.5.0,<2.0.0" redis = { version = ">=2.10.5", optional = true } python-consul = { version = ">=1.0.1", optional = true } @@ -50,7 +50,7 @@ test-filesource = ["pyyaml", "watchdog"] [tool.poetry.group.dev.dependencies] mock = ">=2.0.0" -pytest = ">=2.8" +pytest = "^8.0.0" redis = ">=2.10.5,<5.0.0" boto3 = ">=1.9.71,<2.0.0" coverage = ">=4.4"