diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/utils.py b/packages/pytest-simcore/src/pytest_simcore/helpers/dynamic_scheduler.py similarity index 100% rename from services/dynamic-scheduler/tests/unit/services/generic_scheduler/utils.py rename to packages/pytest-simcore/src/pytest_simcore/helpers/dynamic_scheduler.py diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/core/events.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/core/events.py index b7afd1092240..4e52184d06c2 100644 --- a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/core/events.py +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/core/events.py @@ -28,6 +28,7 @@ from ..services.notifier import get_notifier_lifespans from ..services.rabbitmq import rabbitmq_lifespan from ..services.redis import redis_lifespan +from ..services.scheduler import scheduler_lifespan from ..services.service_tracker import service_tracker_lifespan from ..services.status_monitor import status_monitor_lifespan from .settings import ApplicationSettings @@ -82,6 +83,7 @@ def create_app_lifespan( app_lifespan.add(lifespan) app_lifespan.add(generic_scheduler_lifespan) + app_lifespan.add(scheduler_lifespan) app_lifespan.add(service_tracker_lifespan) app_lifespan.add(deferred_manager_lifespan) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/__init__.py index 5af60ab2ead6..9aed03a902b1 100644 --- a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/__init__.py +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/__init__.py @@ -17,10 +17,12 @@ ) from ._lifespan import generic_scheduler_lifespan from ._models import ( + OperationContext, OperationName, OperationToStart, ProvidedOperationContext, RequiredOperationContext, + ReservedContextKeys, ScheduleId, ) from ._operation import ( @@ -48,6 +50,7 @@ "get_step_store_proxy", "NoDataFoundError", "Operation", + "OperationContext", "OperationContextProxy", "OperationName", "OperationRegistry", @@ -57,6 +60,7 @@ "register_to_start_after_on_executed_completed", "register_to_start_after_on_reverted_completed", "RequiredOperationContext", + "ReservedContextKeys", "restart_operation_step_stuck_during_revert", "restart_operation_step_stuck_in_manual_intervention_during_execute", "ScheduleId", diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_deferred_runner.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_deferred_runner.py index 5fb19777554f..bbdb8f758efb 100644 --- a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_deferred_runner.py +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_deferred_runner.py @@ -5,13 +5,9 @@ from servicelib.deferred_tasks import BaseDeferredHandler, DeferredContext, TaskUID from servicelib.deferred_tasks._models import TaskResultError -from ._errors import ( - OperationContextValueIsNoneError, - ProvidedOperationContextKeysAreMissingError, -) +from ._errors import ProvidedOperationContextKeysAreMissingError from ._event import enqueue_schedule_event from ._models import ( - OperationContext, OperationName, ProvidedOperationContext, ScheduleId, @@ -93,14 +89,7 @@ async def _enqueue_schedule_event_if_group_is_done(context: DeferredContext) -> await enqueue_schedule_event(app, schedule_id) -def _raise_if_any_context_value_is_none( - operation_context: OperationContext, -) -> None: - if any(value is None for value in operation_context.values()): - raise OperationContextValueIsNoneError(operation_context=operation_context) - - -def _raise_if_provided_context_keys_are_missing_or_none( +def _raise_if_provided_context_keys_are_missing( provided_context: ProvidedOperationContext, expected_keys: set[str], ) -> None: @@ -112,8 +101,6 @@ def _raise_if_provided_context_keys_are_missing_or_none( expected_keys=expected_keys, ) - _raise_if_any_context_value_is_none(provided_context) - class DeferredRunner(BaseDeferredHandler[None]): @classmethod @@ -179,26 +166,24 @@ async def run(cls, context: DeferredContext) -> None: required_context = await operation_context_proxy.read( *step.get_execute_requires_context_keys() ) - _raise_if_any_context_value_is_none(required_context) step_provided_operation_context = await step.execute(app, required_context) provided_operation_context = step_provided_operation_context or {} execute_provides_keys = step.get_execute_provides_context_keys() - _raise_if_provided_context_keys_are_missing_or_none( + _raise_if_provided_context_keys_are_missing( provided_operation_context, execute_provides_keys ) else: required_context = await operation_context_proxy.read( *step.get_revert_requires_context_keys() ) - _raise_if_any_context_value_is_none(required_context) step_provided_operation_context = await step.revert(app, required_context) provided_operation_context = step_provided_operation_context or {} revert_provides_keys = step.get_revert_provides_context_keys() - _raise_if_provided_context_keys_are_missing_or_none( + _raise_if_provided_context_keys_are_missing( provided_operation_context, revert_provides_keys ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_errors.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_errors.py index 32357a413b27..c299dcc679d6 100644 --- a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_errors.py +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_errors.py @@ -38,10 +38,6 @@ class UnexpectedStepHandlingError(BaseGenericSchedulerError): ) -class OperationContextValueIsNoneError(BaseGenericSchedulerError): - msg_template: str = "Values of context cannot be None: {operation_context}" - - class ProvidedOperationContextKeysAreMissingError(BaseGenericSchedulerError): msg_template: str = ( "Provided context {provided_context} is missing keys {missing_keys}, was expecting {expected_keys}" diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_event_after_registration.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_event_after_registration.py index bec166efabb5..d57f4c89ae43 100644 --- a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_event_after_registration.py +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/generic_scheduler/_event_after_registration.py @@ -18,6 +18,7 @@ def _get_after_event_manager(app: FastAPI) -> "AfterEventManager": async def register_to_start_after_on_executed_completed( app: FastAPI, schedule_id: ScheduleId, *, to_start: OperationToStart | None ) -> None: + """raises raises NoDataFoundError""" await _get_after_event_manager(app).register_to_start_after( schedule_id, EventType.ON_EXECUTEDD_COMPLETED, to_start=to_start ) @@ -26,6 +27,7 @@ async def register_to_start_after_on_executed_completed( async def register_to_start_after_on_reverted_completed( app: FastAPI, schedule_id: ScheduleId, *, to_start: OperationToStart | None ) -> None: + """raises raises NoDataFoundError""" await _get_after_event_manager(app).register_to_start_after( schedule_id, EventType.ON_REVERT_COMPLETED, to_start=to_start ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/__init__.py new file mode 100644 index 000000000000..7f8c1e60d72c --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/__init__.py @@ -0,0 +1,8 @@ +from ._lifespan import scheduler_lifespan +from ._manager import start_service, stop_service + +__all__: tuple[str, ...] = ( + "scheduler_lifespan", + "start_service", + "stop_service", +) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_errors.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_errors.py new file mode 100644 index 000000000000..c3bba02f277a --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_errors.py @@ -0,0 +1,19 @@ +from common_library.errors_classes import OsparcErrorMixin + + +class BaseSchedulerError(OsparcErrorMixin, Exception): + """base exception for this module""" + + +class UnexpectedCouldNotFindCurrentScheduledIdError(BaseSchedulerError): + msg_template: str = "Could not find current_schedule_id, this is unexpected" + + +class UnexpectedCouldNotFindOperationNameError(BaseSchedulerError): + msg_template: str = "Could not find operation name for schedule_id '{schedule_id}'" + + +class UnexpectedCouldNotDetermineOperationTypeError(BaseSchedulerError): + msg_template: str = ( + "Could not determine operation type from '{operation_name}'. Supported types are {supported_types}" + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_lifespan.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_lifespan.py new file mode 100644 index 000000000000..2900620e4141 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_lifespan.py @@ -0,0 +1,23 @@ +from collections.abc import AsyncIterator + +from fastapi import FastAPI +from fastapi_lifespan_manager import State + +from ...core.settings import ApplicationSettings +from ._operations import registry +from ._redis import RedisStore + + +async def scheduler_lifespan(app: FastAPI) -> AsyncIterator[State]: + settings: ApplicationSettings = app.state.settings + + store = RedisStore(settings.DYNAMIC_SCHEDULER_REDIS) + store.set_to_app_state(app) + + registry.register_operataions() + await store.setup() + + yield {} + + await store.shutdown() + registry.unregister_operations() diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_manager.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_manager.py new file mode 100644 index 000000000000..0f9ef5d4aada --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_manager.py @@ -0,0 +1,174 @@ +import logging +from datetime import timedelta +from typing import Final + +from fastapi import FastAPI +from models_library.api_schemas_dynamic_scheduler.dynamic_services import ( + DynamicServiceStart, + DynamicServiceStop, +) +from models_library.projects_nodes_io import NodeID +from pydantic import NonNegativeFloat +from tenacity import ( + AsyncRetrying, + retry_if_exception_type, + stop_after_delay, + wait_fixed, +) + +from ..generic_scheduler import ( + NoDataFoundError, + OperationToStart, + ScheduleId, + cancel_operation, + get_operation_name_or_none, + register_to_start_after_on_executed_completed, + register_to_start_after_on_reverted_completed, + start_operation, +) +from . import _opration_names +from ._errors import ( + UnexpectedCouldNotFindCurrentScheduledIdError, + UnexpectedCouldNotFindOperationNameError, +) +from ._models import DesiredState, OperationType +from ._redis import RedisServiceStateManager +from ._utils import get_scheduler_operation_type_or_raise + +_logger = logging.getLogger(__name__) + +_WAIT_BETWEEN_RETRIES: Final[NonNegativeFloat] = 0.1 +_MAX_WAIT_TIME_FOR_SCHEDULE_ID: Final[NonNegativeFloat] = timedelta( + seconds=5 +).total_seconds() + + +async def _get_schedule_id_and_opration_type( + app: FastAPI, service_state_manager: RedisServiceStateManager +) -> tuple[ScheduleId, OperationType]: + + # NOTE: current_schedule_id is expected to be None, + # while oprations are switching. + # Waiting a very short time should usually fix the issue. + async for attempt in AsyncRetrying( + wait=wait_fixed(_WAIT_BETWEEN_RETRIES), + stop=stop_after_delay(_MAX_WAIT_TIME_FOR_SCHEDULE_ID), + reraise=True, + retry=retry_if_exception_type(UnexpectedCouldNotFindOperationNameError), + ): + with attempt: + current_schedule_id = await service_state_manager.read( + "current_schedule_id" + ) + if current_schedule_id is None: + raise UnexpectedCouldNotFindCurrentScheduledIdError + + assert current_schedule_id is not None # nosec + + opration_name = await get_operation_name_or_none(app, current_schedule_id) + if opration_name is None: + raise UnexpectedCouldNotFindOperationNameError(schedule_id=current_schedule_id) + + operation_type = get_scheduler_operation_type_or_raise(name=opration_name) + + return current_schedule_id, operation_type + + +async def _switch_to_enforce( + app: FastAPI, schedule_id: ScheduleId, node_id: NodeID +) -> None: + try: + enforce_operation = OperationToStart( + _opration_names.ENFORCE, {"node_id": node_id} + ) + await register_to_start_after_on_executed_completed( + app, schedule_id, to_start=enforce_operation + ) + await register_to_start_after_on_reverted_completed( + app, schedule_id, to_start=enforce_operation + ) + await cancel_operation(app, schedule_id) + except NoDataFoundError: + _logger.debug("Could not switch schedule_id='%s' to ENFORCE.", schedule_id) + + +async def start_service(app: FastAPI, start_data: DynamicServiceStart) -> None: + node_id = start_data.node_uuid + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + + if not await service_state_manager.exists(): + # no data exists, entrypoint for staring the service + await service_state_manager.create_or_update_multiple( + { + "desired_state": DesiredState.RUNNING, + "desired_start_data": start_data, + } + ) + enforce_operation = OperationToStart( + _opration_names.ENFORCE, {"node_id": node_id} + ) + await start_operation( + app, + _opration_names.ENFORCE, + {"node_id": node_id}, + on_execute_completed=enforce_operation, + on_revert_completed=enforce_operation, + ) + _logger.debug("node_di='%s' added to tracking", node_id) + return + + current_schedule_id, operation_type = await _get_schedule_id_and_opration_type( + app, service_state_manager + ) + + match operation_type: + # NOTE: STOP opreration cannot be cancelled + case OperationType.ENFORCE | OperationType.START: + if await service_state_manager.read("current_start_data") != start_data: + await _switch_to_enforce(app, current_schedule_id, node_id) + case OperationType.MONITOR: + await _switch_to_enforce(app, current_schedule_id, node_id) + + # set as current + await service_state_manager.create_or_update("current_start_data", start_data) + + +async def stop_service(app: FastAPI, stop_data: DynamicServiceStop) -> None: + node_id = stop_data.node_id + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + + if not await service_state_manager.exists(): + # it is always possible to schedule the service for a stop, + # primary use case is platform cleanup + await service_state_manager.create_or_update_multiple( + { + "desired_state": DesiredState.STOPPED, + "desired_stop_data": stop_data, + } + ) + enforce_operation = OperationToStart( + _opration_names.ENFORCE, {"node_id": node_id} + ) + await start_operation( + app, + _opration_names.ENFORCE, + {"node_id": node_id}, + on_execute_completed=enforce_operation, + on_revert_completed=enforce_operation, + ) + return + + current_schedule_id, operation_type = await _get_schedule_id_and_opration_type( + app, service_state_manager + ) + + match operation_type: + # NOTE: STOP opreration cannot be cancelled + case OperationType.ENFORCE: + if await service_state_manager.read("current_stop_data") != stop_data: + await _switch_to_enforce(app, current_schedule_id, node_id) + case OperationType.START | OperationType.MONITOR: + await _switch_to_enforce(app, current_schedule_id, node_id) + + # set as current + await service_state_manager.create_or_update("current_stop_data", stop_data) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_models.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_models.py new file mode 100644 index 000000000000..12808caa5fbf --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_models.py @@ -0,0 +1,18 @@ +from enum import auto +from typing import TypeAlias + +from models_library.utils.enums import StrAutoEnum + +SchedulerOperationName: TypeAlias = str + + +class DesiredState(StrAutoEnum): + RUNNING = auto() + STOPPED = auto() + + +class OperationType(StrAutoEnum): + ENFORCE = auto() + START = auto() + MONITOR = auto() + STOP = auto() diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/_common_steps.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/_common_steps.py new file mode 100644 index 000000000000..6337d336d1e7 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/_common_steps.py @@ -0,0 +1,110 @@ +import logging + +from fastapi import FastAPI +from models_library.projects_nodes_io import NodeID + +from ...generic_scheduler import ( + BaseStep, + ProvidedOperationContext, + RequiredOperationContext, + ReservedContextKeys, + ScheduleId, +) +from .._models import DesiredState +from .._redis import RedisServiceStateManager + +_logger = logging.getLogger(__name__) + + +async def _remove_schedule_id(app: FastAPI, *, node_id: NodeID) -> None: + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + await service_state_manager.delete_key("current_schedule_id") + + +class RegisterScheduleId(BaseStep): + @classmethod + def get_execute_requires_context_keys(cls) -> set[str]: + return {ReservedContextKeys.SCHEDULE_ID, "node_id"} + + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + schedule_id: ScheduleId = required_context[ReservedContextKeys.SCHEDULE_ID] + node_id: NodeID = required_context["node_id"] + + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + await service_state_manager.create_or_update("current_schedule_id", schedule_id) + + return None + + @classmethod + def get_revert_requires_context_keys(cls) -> set[str]: + return {"node_id"} + + @classmethod + async def revert( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + await _remove_schedule_id(app, node_id=required_context["node_id"]) + return None + + +class UnRegisterScheduleId(BaseStep): + @classmethod + def get_execute_requires_context_keys(cls) -> set[str]: + return {"node_id"} + + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + await _remove_schedule_id(app, node_id=required_context["node_id"]) + return None + + +class SetCurrentStateRunning(BaseStep): + @classmethod + def get_execute_requires_context_keys(cls) -> set[str]: + return {"node_id"} + + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + node_id: NodeID = required_context["node_id"] + + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + await service_state_manager.create_or_update( + "current_state", DesiredState.RUNNING + ) + + +class SetCurrentStateStopped(BaseStep): + @classmethod + def get_execute_requires_context_keys(cls) -> set[str]: + return {"node_id"} + + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + node_id: NodeID = required_context["node_id"] + + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + await service_state_manager.create_or_update( + "current_state", DesiredState.STOPPED + ) + + +class DoNothing(BaseStep): + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + _ = app + _ = required_context + + _logger.debug("doing nothing, just a placeholder") + + return None diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/enforce.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/enforce.py new file mode 100644 index 000000000000..a7774963058e --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/enforce.py @@ -0,0 +1,141 @@ +import logging + +from fastapi import FastAPI +from models_library.projects_nodes_io import NodeID + +from ...generic_scheduler import ( + BaseStep, + Operation, + OperationContext, + OperationToStart, + ProvidedOperationContext, + RequiredOperationContext, + SingleStepGroup, + start_operation, +) +from .. import _opration_names +from .._models import DesiredState +from .._redis import RedisServiceStateManager +from ._common_steps import RegisterScheduleId, UnRegisterScheduleId + +_logger = logging.getLogger(__name__) + + +class _Prepare(BaseStep): + """ + Figures if a service is legacy or not, + only if it was not previously detenrimined + """ + + @classmethod + def get_execute_requires_context_keys(cls) -> set[str]: + return {"node_id", "is_legacy"} + + @classmethod + def get_execute_provides_context_keys(cls) -> set[str]: + return {"is_legacy"} + + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + node_id: NodeID = required_context["node_id"] + is_legacy: bool | None = required_context["is_legacy"] + + # allows to skip lengthy check + if is_legacy is not None: + return {"is_legacy": is_legacy} + + # TODO: this will be done in a future PR, for now it stays mocked + _ = app + _ = node_id + is_legacy = True + + return {"is_legacy": is_legacy} + + +def _get_start_monitor_stop_initial_context(node_id: NodeID) -> OperationContext: + return {"node_id": node_id} + + +class _Enforce(BaseStep): + @classmethod + def get_execute_requires_context_keys(cls) -> set[str]: + return {"node_id", "is_legacy"} + + @classmethod + async def execute( + cls, app: FastAPI, required_context: RequiredOperationContext + ) -> ProvidedOperationContext | None: + node_id: NodeID = required_context["node_id"] + is_legacy: bool = required_context["is_legacy"] + + service_state_manager = RedisServiceStateManager(app=app, node_id=node_id) + + desired_state = await service_state_manager.read("desired_state") + assert desired_state is not None # nosec + current_state = await service_state_manager.read("current_state") + + monitor_name = ( + _opration_names.LEGACY_MONITOR + if is_legacy + else _opration_names.NEW_STYLE_MONITOR + ) + start_name = ( + _opration_names.LEGACY_START + if is_legacy + else _opration_names.NEW_STYLE_START + ) + stop_name = ( + _opration_names.LEGACY_STOP if is_legacy else _opration_names.NEW_STYLE_STOP + ) + + initial_context = _get_start_monitor_stop_initial_context(node_id) + enforce_operation = OperationToStart( + _opration_names.ENFORCE, initial_context=initial_context + ) + + if current_state == desired_state == DesiredState.RUNNING: + await start_operation( + app, + monitor_name, + initial_context, + on_execute_completed=enforce_operation, + on_revert_completed=enforce_operation, + ) + return None + + if current_state == desired_state == DesiredState.STOPPED: + # do nothing reached the end of everything just remove + await service_state_manager.delete() + _logger.debug("node_di='%s' removed from tracking", node_id) + return None + + match desired_state: + case DesiredState.RUNNING: + await start_operation( + app, + start_name, + initial_context, + on_execute_completed=enforce_operation, + on_revert_completed=enforce_operation, + ) + case DesiredState.STOPPED: + await start_operation( + app, + stop_name, + initial_context, + on_execute_completed=enforce_operation, + on_revert_completed=enforce_operation, + ) + + return None + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + SingleStepGroup(_Prepare), + SingleStepGroup(_Enforce), + SingleStepGroup(UnRegisterScheduleId), + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/__init__.py new file mode 100644 index 000000000000..9945d10ce65d --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/__init__.py @@ -0,0 +1,7 @@ +from . import monitor, start, stop + +__all__: tuple[str, ...] = ( + "monitor", + "start", + "stop", +) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/_steps/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/_steps/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/monitor.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/monitor.py new file mode 100644 index 000000000000..7182712269a3 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/monitor.py @@ -0,0 +1,21 @@ +from datetime import timedelta +from typing import Final + +from ....generic_scheduler import BaseStepGroup, Operation, SingleStepGroup +from .._common_steps import DoNothing, RegisterScheduleId + +_WAIT_BEFORE_REPEAT: Final[timedelta] = timedelta(seconds=5) + + +_steps: list[BaseStepGroup] = [ + SingleStepGroup( + DoNothing, repeat_steps=True, wait_before_repeat=_WAIT_BEFORE_REPEAT + ), +] + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + *_steps, + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/start.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/start.py new file mode 100644 index 000000000000..63aaf148a68c --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/start.py @@ -0,0 +1,17 @@ +from ....generic_scheduler import BaseStepGroup, Operation, SingleStepGroup +from .._common_steps import ( + RegisterScheduleId, + SetCurrentStateRunning, + UnRegisterScheduleId, +) + +_steps: list[BaseStepGroup] = [] + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + *_steps, + SingleStepGroup(SetCurrentStateRunning), + SingleStepGroup(UnRegisterScheduleId), + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/stop.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/stop.py new file mode 100644 index 000000000000..3489537e5f00 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/legacy/stop.py @@ -0,0 +1,18 @@ +from ....generic_scheduler import BaseStepGroup, Operation, SingleStepGroup +from .._common_steps import ( + RegisterScheduleId, + SetCurrentStateStopped, + UnRegisterScheduleId, +) + +_steps: list[BaseStepGroup] = [] + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + *_steps, + SingleStepGroup(SetCurrentStateStopped), + SingleStepGroup(UnRegisterScheduleId), + is_cancellable=False, + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/__init__.py new file mode 100644 index 000000000000..9945d10ce65d --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/__init__.py @@ -0,0 +1,7 @@ +from . import monitor, start, stop + +__all__: tuple[str, ...] = ( + "monitor", + "start", + "stop", +) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/_steps/__init__.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/_steps/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/monitor.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/monitor.py new file mode 100644 index 000000000000..78b8f4353d70 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/monitor.py @@ -0,0 +1,20 @@ +from datetime import timedelta +from typing import Final + +from ....generic_scheduler import BaseStepGroup, Operation, SingleStepGroup +from .._common_steps import DoNothing, RegisterScheduleId + +_WAIT_BEFORE_REPEAT: Final[timedelta] = timedelta(seconds=5) + +_steps: list[BaseStepGroup] = [ + SingleStepGroup( + DoNothing, repeat_steps=True, wait_before_repeat=_WAIT_BEFORE_REPEAT + ), +] + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + *_steps, + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/start.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/start.py new file mode 100644 index 000000000000..63aaf148a68c --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/start.py @@ -0,0 +1,17 @@ +from ....generic_scheduler import BaseStepGroup, Operation, SingleStepGroup +from .._common_steps import ( + RegisterScheduleId, + SetCurrentStateRunning, + UnRegisterScheduleId, +) + +_steps: list[BaseStepGroup] = [] + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + *_steps, + SingleStepGroup(SetCurrentStateRunning), + SingleStepGroup(UnRegisterScheduleId), + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/stop.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/stop.py new file mode 100644 index 000000000000..3489537e5f00 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/new_style/stop.py @@ -0,0 +1,18 @@ +from ....generic_scheduler import BaseStepGroup, Operation, SingleStepGroup +from .._common_steps import ( + RegisterScheduleId, + SetCurrentStateStopped, + UnRegisterScheduleId, +) + +_steps: list[BaseStepGroup] = [] + + +def get_operation() -> Operation: + return Operation( + SingleStepGroup(RegisterScheduleId), + *_steps, + SingleStepGroup(SetCurrentStateStopped), + SingleStepGroup(UnRegisterScheduleId), + is_cancellable=False, + ) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/registry.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/registry.py new file mode 100644 index 000000000000..63e63de74917 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_operations/registry.py @@ -0,0 +1,73 @@ +from typing import Final + +from pydantic import NonNegativeInt + +from ...generic_scheduler import Operation, OperationRegistry, SingleStepGroup +from .. import _opration_names +from . import enforce, legacy, new_style +from ._common_steps import RegisterScheduleId, UnRegisterScheduleId + +_MIN_STEPS_IN_OPERATION: Final[NonNegativeInt] = 3 + + +def _validate_operation(operation: Operation, *, is_monitor: bool) -> None: + min_steps = _MIN_STEPS_IN_OPERATION - 1 if is_monitor else _MIN_STEPS_IN_OPERATION + if len(operation.step_groups) < min_steps: + msg = ( + f"Operation must have at least {min_steps} " + f"startign with {RegisterScheduleId.__name__} and " + f"ending with {UnRegisterScheduleId.__name__}, " + f"got: {operation.step_groups}" + ) + raise ValueError(msg) + first_step_group = operation.step_groups[0] + + if ( + isinstance(first_step_group, SingleStepGroup) + and first_step_group.get_step_subgroup_to_run()[0] is not RegisterScheduleId + ): + msg = ( + f"First step group must be {RegisterScheduleId.__name__}, " + f"got: {first_step_group}" + ) + raise ValueError(msg) + + if is_monitor: + # does not require last step group, since the unregistration of schedule_id + # will be done via RegisterScheduleId's revert + return + + last_step_group = operation.step_groups[-1] + if ( + isinstance(last_step_group, SingleStepGroup) + and last_step_group.get_step_subgroup_to_run()[0] is not UnRegisterScheduleId + ): + msg = ( + f"Last step group must be {UnRegisterScheduleId.__name__}, " + f"got: {last_step_group}" + ) + raise ValueError(msg) + + +def register_operataions() -> None: + for opration_name, operation, is_monitor in ( + (_opration_names.ENFORCE, enforce.get_operation(), False), + (_opration_names.LEGACY_MONITOR, legacy.monitor.get_operation(), True), + (_opration_names.LEGACY_START, legacy.start.get_operation(), False), + (_opration_names.LEGACY_STOP, legacy.stop.get_operation(), False), + (_opration_names.NEW_STYLE_MONITOR, new_style.monitor.get_operation(), True), + (_opration_names.NEW_STYLE_START, new_style.start.get_operation(), False), + (_opration_names.NEW_STYLE_STOP, new_style.start.get_operation(), False), + ): + _validate_operation(operation, is_monitor=is_monitor) + OperationRegistry.register(opration_name, operation) + + +def unregister_operations() -> None: + OperationRegistry.unregister(_opration_names.ENFORCE) + OperationRegistry.unregister(_opration_names.LEGACY_MONITOR) + OperationRegistry.unregister(_opration_names.LEGACY_START) + OperationRegistry.unregister(_opration_names.LEGACY_STOP) + OperationRegistry.unregister(_opration_names.NEW_STYLE_MONITOR) + OperationRegistry.unregister(_opration_names.NEW_STYLE_START) + OperationRegistry.unregister(_opration_names.NEW_STYLE_STOP) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_opration_names.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_opration_names.py new file mode 100644 index 000000000000..8f62c1b184da --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_opration_names.py @@ -0,0 +1,20 @@ +from ._models import OperationType +from ._utils import get_scheduler_oepration_name + +# SHARED + +ENFORCE = get_scheduler_oepration_name(OperationType.ENFORCE, "shared") + + +# NEW STYLE + +NEW_STYLE_START = get_scheduler_oepration_name(OperationType.START, "new_style") +NEW_STYLE_STOP = get_scheduler_oepration_name(OperationType.STOP, "new_style") +NEW_STYLE_MONITOR = get_scheduler_oepration_name(OperationType.MONITOR, "new_style") + + +# LEGACY + +LEGACY_START = get_scheduler_oepration_name(OperationType.START, "legacy") +LEGACY_STOP = get_scheduler_oepration_name(OperationType.STOP, "legacy") +LEGACY_MONITOR = get_scheduler_oepration_name(OperationType.MONITOR, "legacy") diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_redis.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_redis.py new file mode 100644 index 000000000000..e19f4406a484 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_redis.py @@ -0,0 +1,174 @@ +from functools import cached_property +from typing import Any, Final, Literal, NotRequired, TypedDict, overload + +import redis.asyncio as aioredis +from common_library.json_serialization import json_dumps, json_loads +from fastapi import FastAPI +from models_library.api_schemas_dynamic_scheduler.dynamic_services import ( + DynamicServiceStart, + DynamicServiceStop, +) +from models_library.projects_nodes_io import NodeID +from servicelib.fastapi.app_state import SingletonInAppStateMixin +from servicelib.redis._client import RedisClientSDK +from servicelib.redis._utils import handle_redis_returns_union_types +from settings_library.redis import RedisDatabase, RedisSettings + +from ..generic_scheduler import ScheduleId +from ._models import DesiredState + +_SERVICE_STATE_NAMESPACE: Final[str] = "SS" + + +def _get_service_state_key(*, node_id: NodeID) -> str: + # SERVICE_STATE_NAMESPACE:NODE_ID + # - SERVICE_STATE_NAMESPACE: namespace prefix + # - NODE_ID: the unique node_id of the service + # Example: + # - SCH:00000000-0000-0000-0000-000000000000 + return f"{_SERVICE_STATE_NAMESPACE}:{node_id}" + + +class RedisStore(SingletonInAppStateMixin): + app_state_name: str = "scheduler_redis_store" + + def __init__(self, redis_settings: RedisSettings) -> None: + self.redis_settings = redis_settings + + self._client: RedisClientSDK | None = None + + async def setup(self) -> None: + self._client = RedisClientSDK( + self.redis_settings.build_redis_dsn(RedisDatabase.DYNAMIC_SERVICES), + client_name=__name__, + ) + await self._client.setup() + + async def shutdown(self) -> None: + if self._client: + await self._client.shutdown() + + @property + def redis(self) -> aioredis.Redis: + assert self._client # nosec + return self._client.redis + + +class _UpdateServiceStateDict(TypedDict): + desired_state: NotRequired[DesiredState] + desired_start_data: NotRequired[DynamicServiceStart] + desired_stop_data: NotRequired[DynamicServiceStop] + + current_state: NotRequired[DesiredState] + current_start_data: NotRequired[DynamicServiceStart] + current_stop_data: NotRequired[DynamicServiceStop] + + current_schedule_id: NotRequired[ScheduleId] + + +_AllowedDeleteKeys = Literal["current_schedule_id"] + + +class RedisServiceStateManager: + def __init__(self, *, app: FastAPI, node_id: NodeID) -> None: + self.resis_store = RedisStore.get_from_app_state(app) + self.node_id = node_id + + @cached_property + def redis(self) -> aioredis.Redis: + return self.resis_store.redis + + @cached_property + def redis_key(self) -> str: + return _get_service_state_key(node_id=self.node_id) + + @overload + async def create_or_update( + self, key: Literal["desired_state"], value: DesiredState + ) -> None: ... + @overload + async def create_or_update( + self, key: Literal["desired_start_data"], value: DynamicServiceStart + ) -> None: ... + @overload + async def create_or_update( + self, key: Literal["desired_stop_data"], value: DynamicServiceStop + ) -> None: ... + @overload + async def create_or_update( + self, key: Literal["current_state"], value: DesiredState + ) -> None: ... + @overload + async def create_or_update( + self, key: Literal["current_start_data"], value: DynamicServiceStart + ) -> None: ... + @overload + async def create_or_update( + self, key: Literal["current_stop_data"], value: DynamicServiceStop + ) -> None: ... + @overload + async def create_or_update( + self, key: Literal["current_schedule_id"], value: ScheduleId + ) -> None: ... + async def create_or_update(self, key: str, value: Any) -> None: + await handle_redis_returns_union_types( + self.redis.hset(self.redis_key, mapping={key: json_dumps(value)}) + ) + + async def create_or_update_multiple(self, updates: _UpdateServiceStateDict) -> None: + await handle_redis_returns_union_types( + self.redis.hset( + self.redis_key, mapping={k: json_dumps(v) for k, v in updates.items()} + ) + ) + + @overload + async def read(self, key: Literal["desired_state"]) -> DesiredState | None: ... + @overload + async def read( + self, key: Literal["desired_start_data"] + ) -> DynamicServiceStart | None: ... + @overload + async def read( + self, key: Literal["desired_stop_data"] + ) -> DynamicServiceStop | None: ... + @overload + async def read(self, key: Literal["current_state"]) -> DesiredState | None: ... + @overload + async def read( + self, key: Literal["current_start_data"] + ) -> DynamicServiceStart | None: ... + @overload + async def read( + self, key: Literal["current_stop_data"] + ) -> DynamicServiceStop | None: ... + @overload + async def read(self, key: Literal["current_schedule_id"]) -> ScheduleId | None: ... + async def read(self, key: str) -> Any: + list_result: list[str | None] = await handle_redis_returns_union_types( + self.redis.hmget(self.redis_key, [key]) + ) + serialised_result = list_result[0] + if serialised_result is None: + return None + result = json_loads(serialised_result) + + match key: + case "current_start_data" | "desired_start_data": + return DynamicServiceStart.model_validate(result) + case "current_stop_data" | "desired_stop_data": + return DynamicServiceStop.model_validate(result) + case _: + return result + + async def delete_key(self, key: _AllowedDeleteKeys) -> None: + await handle_redis_returns_union_types(self.redis.hdel(self.redis_key, key)) + + async def exists(self) -> bool: + result: int = await handle_redis_returns_union_types( + self.redis.exists(self.redis_key) + ) + return result == 1 + + async def delete(self) -> None: + await handle_redis_returns_union_types(self.redis.delete(self.redis_key)) diff --git a/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_utils.py b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_utils.py new file mode 100644 index 000000000000..dc2a66d49044 --- /dev/null +++ b/services/dynamic-scheduler/src/simcore_service_dynamic_scheduler/services/scheduler/_utils.py @@ -0,0 +1,28 @@ +from typing import Final + +from pydantic import NonNegativeInt + +from ._errors import UnexpectedCouldNotDetermineOperationTypeError +from ._models import OperationType, SchedulerOperationName + +_MIN_STEPS_IN_OPERATION: Final[NonNegativeInt] = 3 + + +def get_scheduler_oepration_name( + operation_type: OperationType, suffix: str +) -> SchedulerOperationName: + return SchedulerOperationName(f"{operation_type.value}_{suffix}") + + +def get_scheduler_operation_type_or_raise( + *, + name: SchedulerOperationName, +) -> OperationType: + operation_type = name.split("_") + try: + return OperationType(operation_type[:1][0]) + except ValueError as exc: + # NOTE: if this is raised there is an actual issue with the operation name + raise UnexpectedCouldNotDetermineOperationTypeError( + operation_name=name, supported_types={x.value for x in OperationType} + ) from exc diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__core.py b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__core.py index 0816e1b8405e..2943464d876e 100644 --- a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__core.py +++ b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__core.py @@ -16,6 +16,17 @@ from asyncpg import NoDataFoundError from fastapi import FastAPI from pydantic import NonNegativeInt, TypeAdapter +from pytest_simcore.helpers.dynamic_scheduler import ( + EXECUTED, + REVERTED, + BaseExpectedStepOrder, + ExecuteRandom, + ExecuteSequence, + RevertRandom, + RevertSequence, + ensure_expected_order, + ensure_keys_in_store, +) from pytest_simcore.helpers.typing_env import EnvVarsDict from servicelib.utils import limited_gather from settings_library.rabbit import RabbitSettings @@ -47,7 +58,6 @@ from simcore_service_dynamic_scheduler.services.generic_scheduler._errors import ( CannotCancelWhileWaitingForManualInterventionError, InitialOperationContextKeyNotAllowedError, - OperationContextValueIsNoneError, OperationNotCancellableError, ProvidedOperationContextKeysAreMissingError, StepNameNotInCurrentGroupError, @@ -63,17 +73,6 @@ stop_after_delay, wait_fixed, ) -from utils import ( - EXECUTED, - REVERTED, - BaseExpectedStepOrder, - ExecuteRandom, - ExecuteSequence, - RevertRandom, - RevertSequence, - ensure_expected_order, - ensure_keys_in_store, -) pytest_simcore_core_services_selection = [ "rabbit", @@ -97,6 +96,7 @@ @pytest.fixture def app_environment( + disable_scheduler_lifespan: None, disable_postgres_lifespan: None, disable_service_tracker_lifespan: None, disable_notifier_lifespan: None, @@ -1504,7 +1504,6 @@ async def test_operation_context_usage( await ensure_keys_in_store(selected_app, expected_keys=set()) - assert f"{OperationContextValueIsNoneError.__name__}" not in caplog.text assert f"{ProvidedOperationContextKeysAreMissingError.__name__}" not in caplog.text @@ -1557,62 +1556,6 @@ async def test_operation_initial_context_using_key_provided_by_step( await ensure_keys_in_store(selected_app, expected_keys=set()) -@pytest.mark.parametrize("app_count", [10]) -@pytest.mark.parametrize( - "operation, initial_context, expected_order", - [ - pytest.param( - Operation( - SingleStepGroup(RPCtxS1), - ), - { - # `bs__c_req_1` is missing - }, - [ - RevertSequence(RPCtxS1), - ], - id="missing_context_key", - ), - pytest.param( - Operation( - SingleStepGroup(RPCtxS1), - ), - { - "bs__e_req_1": None, - }, - [ - RevertSequence(RPCtxS1), - ], - id="context_key_is_none", - ), - ], -) -async def test_step_does_not_receive_context_key_or_is_none( - preserve_caplog_for_async_logging: None, - caplog: pytest.LogCaptureFixture, - steps_call_order: list[tuple[str, str]], - selected_app: FastAPI, - register_operation: Callable[[OperationName, Operation], None], - operation: Operation, - operation_name: OperationName, - initial_context: OperationContext, - expected_order: list[BaseExpectedStepOrder], -): - caplog.at_level(logging.DEBUG) - caplog.clear() - - register_operation(operation_name, operation) - - schedule_id = await start_operation(selected_app, operation_name, initial_context) - assert TypeAdapter(ScheduleId).validate_python(schedule_id) - - await _ensure_log_mesage(caplog, message=OperationContextValueIsNoneError.__name__) - - await ensure_expected_order(steps_call_order, expected_order) - - await ensure_keys_in_store(selected_app, expected_keys=set()) - - class _BadImplementedStep(BaseStep): @classmethod def _get_provided_context( @@ -1675,32 +1618,6 @@ async def revert( @pytest.mark.parametrize( "operation, initial_context, expected_error_str, expected_order, expected_keys", [ - pytest.param( - Operation( - SingleStepGroup(_BadImplementedStep), - ), - { - "trigger_revert": False, - "to_return": { - "add_to_return": True, - "keys": {"a_key": None}, - }, - }, - f"{OperationContextValueIsNoneError.__name__}: Values of context cannot be None: {{'a_key'", - [ - ExecuteSequence(_BadImplementedStep), - RevertSequence(_BadImplementedStep), - ], - { - "SCH:{schedule_id}", - "SCH:{schedule_id}:GROUPS:test_op:0S:E", - "SCH:{schedule_id}:GROUPS:test_op:0S:R", - "SCH:{schedule_id}:OP_CTX:test_op", - "SCH:{schedule_id}:STEPS:test_op:0S:E:_BadImplementedStep", - "SCH:{schedule_id}:STEPS:test_op:0S:R:_BadImplementedStep", - }, - id="execute-returns-key-set-to-None", - ), pytest.param( Operation( SingleStepGroup(_BadImplementedStep), @@ -1726,32 +1643,6 @@ async def revert( }, id="execute-does-not-set-the-key-to-return", ), - pytest.param( - Operation( - SingleStepGroup(_BadImplementedStep), - ), - { - "trigger_revert": True, - "to_return": { - "add_to_return": True, - "keys": {"a_key": None}, - }, - }, - f"{OperationContextValueIsNoneError.__name__}: Values of context cannot be None: {{'a_key'", - [ - ExecuteSequence(_BadImplementedStep), - RevertSequence(_BadImplementedStep), - ], - { - "SCH:{schedule_id}", - "SCH:{schedule_id}:GROUPS:test_op:0S:E", - "SCH:{schedule_id}:GROUPS:test_op:0S:R", - "SCH:{schedule_id}:OP_CTX:test_op", - "SCH:{schedule_id}:STEPS:test_op:0S:E:_BadImplementedStep", - "SCH:{schedule_id}:STEPS:test_op:0S:R:_BadImplementedStep", - }, - id="revert-returns-key-set-to-None", - ), pytest.param( Operation( SingleStepGroup(_BadImplementedStep), diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__deferred_runner.py b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__deferred_runner.py index d0c9f8853b08..2bd5bc59a00d 100644 --- a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__deferred_runner.py +++ b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__deferred_runner.py @@ -59,6 +59,7 @@ @pytest.fixture def app_environment( + disable_scheduler_lifespan: None, disable_postgres_lifespan: None, disable_service_tracker_lifespan: None, disable_notifier_lifespan: None, diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_after.py b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_after.py index 33fdc7ddf373..b92f2ed1cab4 100644 --- a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_after.py +++ b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_after.py @@ -11,6 +11,7 @@ from fastapi import FastAPI from pydantic import TypeAdapter from pytest_mock import MockerFixture +from pytest_simcore.helpers.dynamic_scheduler import ensure_keys_in_store from pytest_simcore.helpers.typing_env import EnvVarsDict from settings_library.rabbit import RabbitSettings from settings_library.redis import RedisSettings @@ -39,7 +40,6 @@ OperationEventsProxy, Store, ) -from utils import ensure_keys_in_store pytest_simcore_core_services_selection = [ "rabbit", @@ -52,6 +52,7 @@ @pytest.fixture def app_environment( + disable_scheduler_lifespan: None, disable_postgres_lifespan: None, disable_service_tracker_lifespan: None, disable_notifier_lifespan: None, diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_scheduler.py b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_scheduler.py index 3851e1877f05..3a0283901261 100644 --- a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_scheduler.py +++ b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__event_scheduler.py @@ -58,6 +58,7 @@ def disable_other_generic_scheduler_modules(mocker: MockerFixture) -> None: @pytest.fixture def app_environment( + disable_scheduler_lifespan: None, disable_other_generic_scheduler_modules: None, disable_redis_lifespan: None, disable_postgres_lifespan: None, diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__store.py b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__store.py index c22ced8f5f5d..b23b27ff467d 100644 --- a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__store.py +++ b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test__store.py @@ -355,6 +355,9 @@ async def test_operation_context_proxy( assert await proxy.read(*provided_context.keys()) == provided_context + # if a keys is missing the value is always None + assert await proxy.read("missing-key") == {"missing-key": None} + async def test_operation_removal_proxy(store: Store, schedule_id: ScheduleId): await _assert_keys(store, set()) diff --git a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test_generic_scheduler.py b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test_generic_scheduler.py index f22dee36bbfb..b0b057481c98 100644 --- a/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test_generic_scheduler.py +++ b/services/dynamic-scheduler/tests/unit/services/generic_scheduler/test_generic_scheduler.py @@ -19,6 +19,14 @@ from common_library.async_tools import cancel_wait_task from fastapi import FastAPI from pydantic import NonNegativeFloat, NonNegativeInt +from pytest_simcore.helpers.dynamic_scheduler import ( + BaseExpectedStepOrder, + ExecuteRandom, + ExecuteSequence, + RevertSequence, + ensure_expected_order, + ensure_keys_in_store, +) from pytest_simcore.helpers.paused_container import pause_rabbit, pause_redis from pytest_simcore.helpers.typing_env import EnvVarsDict from servicelib.deferred_tasks import DeferredContext @@ -46,14 +54,6 @@ from simcore_service_dynamic_scheduler.services.generic_scheduler._errors import ( OperationInitialContextKeyNotFoundError, ) -from utils import ( - BaseExpectedStepOrder, - ExecuteRandom, - ExecuteSequence, - RevertSequence, - ensure_expected_order, - ensure_keys_in_store, -) pytest_simcore_core_services_selection = [ "rabbit", @@ -80,6 +80,7 @@ def _get_random_interruption_duration() -> NonNegativeFloat: @pytest.fixture def app_environment( + disable_scheduler_lifespan: None, disable_postgres_lifespan: None, disable_service_tracker_lifespan: None, disable_notifier_lifespan: None, diff --git a/services/dynamic-scheduler/tests/unit/services/scheduler/_oprations/test_registry.py b/services/dynamic-scheduler/tests/unit/services/scheduler/_oprations/test_registry.py new file mode 100644 index 000000000000..cb51f89bea48 --- /dev/null +++ b/services/dynamic-scheduler/tests/unit/services/scheduler/_oprations/test_registry.py @@ -0,0 +1,22 @@ +# pylint:disable=protected-access + +from pydantic import NonNegativeInt +from simcore_service_dynamic_scheduler.services.generic_scheduler import ( + OperationRegistry, +) +from simcore_service_dynamic_scheduler.services.scheduler._operations.registry import ( + register_operataions, + unregister_operations, +) + + +def _ensure_registered_operations(*, count: NonNegativeInt) -> None: + assert len(OperationRegistry._OPERATIONS) == count # noqa: SLF001 + + +def test_register_unregister_operations() -> None: + _ensure_registered_operations(count=0) + register_operataions() + _ensure_registered_operations(count=7) + unregister_operations() + _ensure_registered_operations(count=0) diff --git a/services/dynamic-scheduler/tests/unit/services/scheduler/test__redis.py b/services/dynamic-scheduler/tests/unit/services/scheduler/test__redis.py new file mode 100644 index 000000000000..6829dfb17153 --- /dev/null +++ b/services/dynamic-scheduler/tests/unit/services/scheduler/test__redis.py @@ -0,0 +1,150 @@ +# pylint:disable=redefined-outer-name +# pylint:disable=unused-argument + + +import pytest +from faker import Faker +from fastapi import FastAPI +from models_library.api_schemas_dynamic_scheduler.dynamic_services import ( + DynamicServiceStart, + DynamicServiceStop, +) +from models_library.projects_nodes_io import NodeID +from pydantic import TypeAdapter +from pytest_simcore.helpers.typing_env import EnvVarsDict +from settings_library.redis import RedisSettings +from simcore_service_dynamic_scheduler.services.generic_scheduler import ScheduleId +from simcore_service_dynamic_scheduler.services.scheduler._models import DesiredState +from simcore_service_dynamic_scheduler.services.scheduler._redis import ( + RedisServiceStateManager, +) + +pytest_simcore_core_services_selection = [ + "redis", +] +pytest_simcore_ops_services_selection = [ + "redis-commander", +] + + +@pytest.fixture +def app_environment( + disable_deferred_manager_lifespan: None, + disable_rabbitmq_lifespan: None, + disable_generic_scheduler_lifespan: None, + disable_postgres_lifespan: None, + disable_service_tracker_lifespan: None, + disable_notifier_lifespan: None, + disable_status_monitor_lifespan: None, + app_environment: EnvVarsDict, + redis_service: RedisSettings, + remove_redis_data: None, +) -> EnvVarsDict: + return app_environment + + +@pytest.fixture +def node_id(faker: Faker) -> NodeID: + return faker.uuid4(cast_to=None) + + +@pytest.fixture +def schedule_id(faker: Faker) -> ScheduleId: + return faker.uuid4() + + +@pytest.fixture +def dynamic_service_start() -> DynamicServiceStart: + return TypeAdapter(DynamicServiceStart).validate_python( + DynamicServiceStart.model_json_schema()["example"] + ) + + +@pytest.fixture +def dynamic_service_stop() -> DynamicServiceStop: + return TypeAdapter(DynamicServiceStop).validate_python( + DynamicServiceStop.model_json_schema()["example"] + ) + + +async def test_redis_service_state( + app: FastAPI, + node_id: NodeID, + schedule_id: ScheduleId, + dynamic_service_start: DynamicServiceStart, + dynamic_service_stop: DynamicServiceStop, +): + state_manager = RedisServiceStateManager(app=app, node_id=node_id) + + # 1. check nothing present + assert await state_manager.exists() is False + assert await state_manager.read("desired_state") is None + assert await state_manager.read("desired_start_data") is None + assert await state_manager.read("desired_stop_data") is None + assert await state_manager.read("current_start_data") is None + assert await state_manager.read("current_stop_data") is None + assert await state_manager.read("current_schedule_id") is None + # reading does not create items + assert await state_manager.exists() is False + + # 2. create some entries + await state_manager.create_or_update("desired_state", DesiredState.RUNNING) + # already works with one entry regarless of which one is + assert await state_manager.exists() is True + assert await state_manager.read("desired_state") == DesiredState.RUNNING + + await state_manager.create_or_update("current_state", DesiredState.STOPPED) + assert await state_manager.read("current_state") == DesiredState.STOPPED + + await state_manager.create_or_update("current_schedule_id", schedule_id) + assert await state_manager.read("current_schedule_id") == schedule_id + + await state_manager.create_or_update("current_start_data", dynamic_service_start) + assert await state_manager.read("current_start_data") == dynamic_service_start + await state_manager.create_or_update("desired_start_data", dynamic_service_start) + assert await state_manager.read("desired_start_data") == dynamic_service_start + + await state_manager.create_or_update("current_stop_data", dynamic_service_stop) + assert await state_manager.read("current_stop_data") == dynamic_service_stop + await state_manager.create_or_update("desired_stop_data", dynamic_service_stop) + assert await state_manager.read("desired_stop_data") == dynamic_service_stop + # still true regardless of how many entries + assert await state_manager.exists() is True + + # 3. remove nothig is presnet any longer + await state_manager.delete() + assert await state_manager.exists() is False + + # 4 setting multiple is the same + + await state_manager.create_or_update_multiple( + { + "desired_state": DesiredState.STOPPED, + "desired_stop_data": dynamic_service_stop, + "desired_start_data": dynamic_service_start, + "current_schedule_id": schedule_id, + "current_state": DesiredState.STOPPED, + "current_start_data": dynamic_service_start, + "current_stop_data": dynamic_service_stop, + } + ) + assert await state_manager.exists() is True + assert await state_manager.read("desired_state") == DesiredState.STOPPED + assert await state_manager.read("desired_start_data") == dynamic_service_start + assert await state_manager.read("desired_stop_data") == dynamic_service_stop + assert await state_manager.read("current_schedule_id") == schedule_id + assert await state_manager.read("current_state") == DesiredState.STOPPED + assert await state_manager.read("current_start_data") == dynamic_service_start + assert await state_manager.read("current_stop_data") == dynamic_service_stop + + # 5. remove nothig is presnet any longer + await state_manager.delete() + assert await state_manager.exists() is False + + # 6. deleting a key + await state_manager.create_or_update("current_schedule_id", schedule_id) + assert await state_manager.read("current_schedule_id") == schedule_id + await state_manager.delete_key("current_schedule_id") + assert await state_manager.read("current_schedule_id") is None + # can also delete unexising key withtout errors + await state_manager.delete_key("current_schedule_id") diff --git a/services/dynamic-scheduler/tests/unit/services/scheduler/test__utils.py b/services/dynamic-scheduler/tests/unit/services/scheduler/test__utils.py new file mode 100644 index 000000000000..a9b478fb61aa --- /dev/null +++ b/services/dynamic-scheduler/tests/unit/services/scheduler/test__utils.py @@ -0,0 +1,37 @@ +# pylint:disable=redefined-outer-name +# pylint:disable=unused-argument + + +import pytest +from simcore_service_dynamic_scheduler.services.scheduler._errors import ( + UnexpectedCouldNotDetermineOperationTypeError, +) +from simcore_service_dynamic_scheduler.services.scheduler._models import OperationType +from simcore_service_dynamic_scheduler.services.scheduler._utils import ( + get_scheduler_oepration_name, + get_scheduler_operation_type_or_raise, +) + + +@pytest.mark.parametrize( + "suffix", + [ + "1234", + "abcd", + "some_suffix", + "_", + "____", + "___asd___Asd", + "", + ], +) +@pytest.mark.parametrize("operation_type", OperationType) +def test_operation_names(operation_type: OperationType, suffix: str) -> None: + operation_name = get_scheduler_oepration_name(operation_type, suffix) + assert operation_name == f"{operation_type}_{suffix}" + assert get_scheduler_operation_type_or_raise(name=operation_name) == operation_type + + +def test_raise_on_invalid_operation_name() -> None: + with pytest.raises(UnexpectedCouldNotDetermineOperationTypeError): + get_scheduler_operation_type_or_raise(name="invalid_operation_name")