|
1 | | -from typing import TypeAlias |
| 1 | +from typing import Final, Literal, TypeAlias |
| 2 | + |
| 3 | +from dask.typing import Key |
| 4 | +from distributed.scheduler import TaskStateState as SchedulerTaskState |
| 5 | +from distributed.worker_state_machine import TaskStateState as WorkerTaskState |
| 6 | +from models_library.projects_state import RunningState |
| 7 | +from pydantic import BaseModel |
2 | 8 |
|
3 | 9 | DaskJobID: TypeAlias = str |
4 | 10 | DaskResources: TypeAlias = dict[str, int | float] |
| 11 | + |
| 12 | +TASK_LIFE_CYCLE_EVENT: Final[str] = "task-lifecycle-{key}" |
| 13 | +TASK_RUNNING_PROGRESS_EVENT: Final[str] = "task-progress-{key}" |
| 14 | +_SCHEDULER_TASK_STATE_TO_RUNNING_STATE: Final[ |
| 15 | + dict[SchedulerTaskState, RunningState] |
| 16 | +] = { |
| 17 | + "released": RunningState.NOT_STARTED, # Known but not actively computing or in memory |
| 18 | + "waiting": RunningState.PENDING, # On track to be computed, waiting on dependencies to arrive in memory |
| 19 | + "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). |
| 20 | + "queued": RunningState.WAITING_FOR_RESOURCES, # Ready to be computed, but all workers are already full. |
| 21 | + "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). |
| 22 | + "memory": RunningState.SUCCESS, # In memory on one or more workers |
| 23 | + "erred": RunningState.FAILED, # Task computation, or one of its dependencies, has encountered an error |
| 24 | + "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. |
| 25 | +} |
| 26 | + |
| 27 | +_WORKER_TASK_STATE_TO_RUNNING_STATE: Final[dict[WorkerTaskState, RunningState]] = { |
| 28 | + "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. |
| 29 | + "constrained": RunningState.PENDING, # Like ready, but the user specified resource constraints for this task. The task can be found in the WorkerState.constrained queue. |
| 30 | + "error": RunningState.FAILED, # Task execution failed |
| 31 | + "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. |
| 32 | + "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. |
| 33 | + "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. |
| 34 | + "forgotten": RunningState.UNKNOWN, # The scheduler asked this worker to forget about the task, and there are neither dependents nor dependencies on the same worker. |
| 35 | + "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. |
| 36 | + "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. |
| 37 | + "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. |
| 38 | + "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. |
| 39 | + "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. |
| 40 | + "rescheduled": RunningState.PENDING, # The task just raised the Reschedule exception. This is a transitory state, which is not stored permanently. |
| 41 | + "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. |
| 42 | + "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. |
| 43 | +} |
| 44 | + |
| 45 | + |
| 46 | +class TaskLifeCycleState(BaseModel): |
| 47 | + key: str |
| 48 | + source: Literal["scheduler", "worker"] |
| 49 | + worker: str | None |
| 50 | + state: RunningState |
| 51 | + |
| 52 | + @classmethod |
| 53 | + def from_scheduler_task_state( |
| 54 | + cls, key: Key, worker: str | None, task_state: SchedulerTaskState |
| 55 | + ) -> "TaskLifeCycleState": |
| 56 | + return cls( |
| 57 | + key=f"{key!r}", |
| 58 | + source="scheduler", |
| 59 | + worker=worker, |
| 60 | + state=_SCHEDULER_TASK_STATE_TO_RUNNING_STATE[task_state], |
| 61 | + ) |
| 62 | + |
| 63 | + @classmethod |
| 64 | + def from_worker_task_state( |
| 65 | + cls, key: Key, worker: str | None, task_state: WorkerTaskState |
| 66 | + ) -> "TaskLifeCycleState": |
| 67 | + return cls( |
| 68 | + key=f"{key!r}", |
| 69 | + source="worker", |
| 70 | + worker=worker, |
| 71 | + state=_WORKER_TASK_STATE_TO_RUNNING_STATE[task_state], |
| 72 | + ) |
0 commit comments