-
Notifications
You must be signed in to change notification settings - Fork 32
✨🐛Computational Backend: Introduce Dask plugins for tasks lifecycle #7686
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
sanderegg
merged 46 commits into
ITISFoundation:master
from
sanderegg:dask-sidecar/task-lifecycle-events
May 20, 2025
Merged
Changes from all commits
Commits
Show all changes
46 commits
Select commit
Hold shift + click to select a range
f5acf41
now we also test the scheduler preload
sanderegg 6963a8c
improve logs
sanderegg 4ca2bd3
show manual intervention needed if started since >2 minutes
sanderegg 2107619
add some better reason
sanderegg 6224bf8
added life cycle plugins
sanderegg 8d302cb
renamed
sanderegg 937143a
refactor
sanderegg 4014d3d
ongoing
sanderegg 9f5d9aa
sending events about task status
sanderegg a451c39
check returned states
sanderegg a3a3cbf
rename
sanderegg f46aff5
mypy
sanderegg 38c0a7a
seems to work
sanderegg ced8860
added get_task_status2
sanderegg 93dc096
ongoing removal of old stuff
sanderegg c1ea3f8
ongoing
sanderegg b63f4fa
removed daskstate and upgrade test using running state
sanderegg 75768fa
release constraint
sanderegg f6ea7c5
added dependency to dask-sidecar
sanderegg 08438b9
removed old parts
sanderegg a62fd7f
ensure plugins are loaded for tests
sanderegg 0f8b59d
added dependency
sanderegg 63ee2c1
allows to setup plugins
sanderegg 3f40d64
make the call async
sanderegg 4d6ea9c
the test now runs
sanderegg 7e0faa6
set a timeout
sanderegg 5cb2fc0
mypy
sanderegg 65bb64e
moved task life cycle plugins to dask-task-library
sanderegg 65262ca
removed dependency to dask-sidecar
sanderegg d5e4314
fixed paths
sanderegg c9b3698
moved plugins down one level
sanderegg 4084721
manage to poll task progress now
sanderegg 3e3aab1
revert to have events
sanderegg eb4debb
update notes
sanderegg 205c968
added specific events to be more effective
sanderegg 443f596
remove servicelib from dask-task-models
sanderegg 27d832c
added dependency for testing
sanderegg 4ec4d40
mypy
sanderegg c8bec8f
fixed tests
sanderegg 15b135e
pylint
sanderegg ffbc5a9
fix difference
sanderegg 1f0ac5f
language
sanderegg 23c1fcb
pylint
sanderegg 3230606
pylint
sanderegg ab92c70
timeout increase
sanderegg 1567bdd
revert changes
sanderegg File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
70 changes: 69 additions & 1 deletion
70
packages/dask-task-models-library/src/dask_task_models_library/models.py
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,4 +1,72 @@ | ||
| from typing import TypeAlias | ||
| from typing import Final, Literal, TypeAlias | ||
|
|
||
| from dask.typing import Key | ||
| from distributed.scheduler import TaskStateState as SchedulerTaskState | ||
| from distributed.worker_state_machine import TaskStateState as WorkerTaskState | ||
| from models_library.projects_state import RunningState | ||
| from pydantic import BaseModel | ||
|
|
||
| DaskJobID: TypeAlias = str | ||
| DaskResources: TypeAlias = dict[str, int | float] | ||
|
|
||
| TASK_LIFE_CYCLE_EVENT: Final[str] = "task-lifecycle-{key}" | ||
| TASK_RUNNING_PROGRESS_EVENT: Final[str] = "task-progress-{key}" | ||
| _SCHEDULER_TASK_STATE_TO_RUNNING_STATE: Final[ | ||
| dict[SchedulerTaskState, RunningState] | ||
| ] = { | ||
| "released": RunningState.NOT_STARTED, # Known but not actively computing or in memory | ||
| "waiting": RunningState.PENDING, # On track to be computed, waiting on dependencies to arrive in memory | ||
| "no-worker": RunningState.WAITING_FOR_RESOURCES, # Ready to be computed, but no appropriate worker exists (for example because of resource restrictions, or because no worker is connected at all). | ||
| "queued": RunningState.WAITING_FOR_RESOURCES, # Ready to be computed, but all workers are already full. | ||
| "processing": RunningState.PENDING, # All dependencies are available and the task is assigned to a worker for compute (the scheduler doesn’t know whether it’s in a worker queue or actively being computed). | ||
| "memory": RunningState.SUCCESS, # In memory on one or more workers | ||
| "erred": RunningState.FAILED, # Task computation, or one of its dependencies, has encountered an error | ||
| "forgotten": RunningState.UNKNOWN, # Task is no longer needed by any client or dependent task, so it disappears from the scheduler as well. As soon as a task reaches this state, it is immediately dereferenced from the scheduler. | ||
| } | ||
|
|
||
| _WORKER_TASK_STATE_TO_RUNNING_STATE: Final[dict[WorkerTaskState, RunningState]] = { | ||
| "cancelled": RunningState.ABORTED, # The scheduler asked to forget about this task, but it’s technically impossible at the moment. See Task cancellation. The task can be found in whatever collections it was in its previous state. | ||
| "constrained": RunningState.PENDING, # Like ready, but the user specified resource constraints for this task. The task can be found in the WorkerState.constrained queue. | ||
| "error": RunningState.FAILED, # Task execution failed | ||
| "executing": RunningState.STARTED, # The task is currently being computed on a thread. It can be found in the WorkerState.executing set and in the distributed.worker.Worker.active_threads dict. | ||
| "fetch": RunningState.PENDING, # This task is in memory on one or more peer workers, but not on this worker. Its data is queued to be transferred over the network, either because it’s a dependency of a task in waiting state, or because the Active Memory Manager requested it to be replicated here. The task can be found in the WorkerState.data_needed heap. | ||
| "flight": RunningState.PENDING, # The task data is currently being transferred over the network from another worker. The task can be found in the WorkerState.in_flight_tasks and WorkerState.in_flight_workers collections. | ||
| "forgotten": RunningState.UNKNOWN, # The scheduler asked this worker to forget about the task, and there are neither dependents nor dependencies on the same worker. | ||
| "long-running": RunningState.STARTED, # Like executing, but the user code called distributed.secede() so the task no longer counts towards the maximum number of concurrent tasks. It can be found in the WorkerState.long_running set and in the distributed.worker.Worker.active_threads dict. | ||
| "memory": RunningState.SUCCESS, # Task execution completed, or the task was successfully transferred from another worker, and is now held in either WorkerState.data or WorkerState.actors. | ||
| "missing": RunningState.PENDING, # Like fetch, but all peer workers that were listed by the scheduler are either unreachable or have responded they don’t actually have the task data. The worker will periodically ask the scheduler if it knows of additional replicas; when it does, the task will transition again to fetch. The task can be found in the WorkerState.missing_dep_flight set. | ||
| "ready": RunningState.PENDING, # The task is ready to be computed; all of its dependencies are in memory on the current worker and it’s waiting for an available thread. The task can be found in the WorkerState.ready heap. | ||
| "released": RunningState.PENDING, # Known but not actively computing or in memory. A task can stay in this state when the scheduler asked to forget it, but it has dependent tasks on the same worker. | ||
| "rescheduled": RunningState.PENDING, # The task just raised the Reschedule exception. This is a transitory state, which is not stored permanently. | ||
| "resumed": RunningState.PENDING, # The task was recovered from cancelled state. See Task cancellation. The task can be found in whatever collections it was in its previous state. | ||
| "waiting": RunningState.PENDING, # The scheduler has added the task to the worker queue. All of its dependencies are in memory somewhere on the cluster, but not all of them are in memory on the current worker, so they need to be fetched. | ||
| } | ||
|
|
||
|
|
||
| class TaskLifeCycleState(BaseModel): | ||
| key: str | ||
| source: Literal["scheduler", "worker"] | ||
| worker: str | None | ||
| state: RunningState | ||
|
|
||
| @classmethod | ||
| def from_scheduler_task_state( | ||
| cls, key: Key, worker: str | None, task_state: SchedulerTaskState | ||
| ) -> "TaskLifeCycleState": | ||
| return cls( | ||
| key=f"{key!r}", | ||
| source="scheduler", | ||
| worker=worker, | ||
| state=_SCHEDULER_TASK_STATE_TO_RUNNING_STATE[task_state], | ||
| ) | ||
|
|
||
| @classmethod | ||
| def from_worker_task_state( | ||
| cls, key: Key, worker: str | None, task_state: WorkerTaskState | ||
| ) -> "TaskLifeCycleState": | ||
| return cls( | ||
| key=f"{key!r}", | ||
| source="worker", | ||
| worker=worker, | ||
| state=_WORKER_TASK_STATE_TO_RUNNING_STATE[task_state], | ||
| ) | ||
Empty file.
54 changes: 54 additions & 0 deletions
54
...k-models-library/src/dask_task_models_library/plugins/task_life_cycle_scheduler_plugin.py
sanderegg marked this conversation as resolved.
Show resolved
Hide resolved
|
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,54 @@ | ||
| # pylint: disable=unused-argument | ||
| import logging | ||
| from typing import Any | ||
|
|
||
| import click | ||
| from dask.typing import Key | ||
| from distributed import Scheduler, SchedulerPlugin | ||
| from distributed.scheduler import TaskStateState | ||
|
|
||
| from ..models import TASK_LIFE_CYCLE_EVENT, TaskLifeCycleState | ||
|
|
||
| _logger = logging.getLogger(__name__) | ||
|
|
||
|
|
||
| class TaskLifecycleSchedulerPlugin(SchedulerPlugin): | ||
sanderegg marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| def __init__(self) -> None: | ||
| self.scheduler = None | ||
| _logger.info("initialized TaskLifecycleSchedulerPlugin") | ||
|
|
||
| async def start(self, scheduler: Scheduler) -> None: | ||
| self.scheduler = scheduler # type: ignore[assignment] | ||
| _logger.info("started TaskLifecycleSchedulerPlugin") | ||
|
|
||
| def transition( | ||
| self, | ||
| key: Key, | ||
| start: TaskStateState, | ||
| finish: TaskStateState, | ||
| *args: Any, # noqa: ARG002 | ||
| stimulus_id: str, | ||
| **kwargs: Any, | ||
| ): | ||
| _logger.debug( | ||
| "Task %s transition from %s to %s due to %s", | ||
| key, | ||
| start, | ||
| finish, | ||
| stimulus_id, | ||
| ) | ||
|
|
||
| assert self.scheduler # nosec | ||
|
|
||
| self.scheduler.log_event( | ||
| TASK_LIFE_CYCLE_EVENT.format(key=key), | ||
| TaskLifeCycleState.from_scheduler_task_state( | ||
| key, kwargs.get("worker"), finish | ||
| ).model_dump(mode="json"), | ||
| ) | ||
|
|
||
|
|
||
| @click.command() | ||
| def dask_setup(scheduler): | ||
| plugin = TaskLifecycleSchedulerPlugin() | ||
| scheduler.add_plugin(plugin) | ||
48 changes: 48 additions & 0 deletions
48
...task-models-library/src/dask_task_models_library/plugins/task_life_cycle_worker_plugin.py
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,48 @@ | ||
| import logging | ||
| from collections.abc import Awaitable | ||
| from typing import Any | ||
|
|
||
| import click | ||
| from dask.typing import Key | ||
| from distributed import WorkerPlugin | ||
| from distributed.worker import Worker | ||
| from distributed.worker_state_machine import TaskStateState | ||
|
|
||
| from ..models import TASK_LIFE_CYCLE_EVENT, TaskLifeCycleState | ||
|
|
||
| _logger = logging.getLogger(__name__) | ||
|
|
||
|
|
||
| class TaskLifecycleWorkerPlugin(WorkerPlugin): | ||
| def __init__(self) -> None: | ||
| self._worker = None | ||
| _logger.info("TaskLifecycleWorkerPlugin initialized") | ||
|
|
||
| def setup(self, worker: Worker) -> Awaitable[None]: | ||
| async def _() -> None: | ||
| self._worker = worker # type: ignore[assignment] | ||
| _logger.info("TaskLifecycleWorkerPlugin setup completed") | ||
|
|
||
| return _() | ||
|
|
||
| def transition( | ||
| self, | ||
| key: Key, | ||
| start: TaskStateState, | ||
| finish: TaskStateState, | ||
| **kwargs: Any, | ||
| ): | ||
| _logger.info("Task '%s' transition from %s to %s", key, start, finish) | ||
| assert self._worker # nosec | ||
| self._worker.log_event( | ||
| TASK_LIFE_CYCLE_EVENT.format(key=key), | ||
| TaskLifeCycleState.from_worker_task_state( | ||
| key, kwargs.get("worker"), finish | ||
| ).model_dump(mode="json"), | ||
| ) | ||
|
|
||
|
|
||
| @click.command() | ||
| async def dask_setup(worker: Worker) -> None: | ||
| plugin = TaskLifecycleWorkerPlugin() | ||
| await worker.plugin_add(plugin) | ||
sanderegg marked this conversation as resolved.
Show resolved
Hide resolved
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.