from __future__ import annotations
import asyncio
import contextlib
import heapq
import inspect
import itertools
import json
import logging
import math
import operator
import os
import pickle
import random
import sys
import uuid
import warnings
import weakref
from collections import defaultdict, deque
from collections.abc import (
Callable,
Collection,
Container,
Hashable,
Iterable,
Iterator,
Mapping,
Set,
)
from contextlib import suppress
from functools import partial
from numbers import Number
from typing import Any, ClassVar, Literal, cast
import psutil
from sortedcontainers import SortedDict, SortedSet
from tlz import (
first,
groupby,
merge,
merge_sorted,
merge_with,
partition,
pluck,
second,
valmap,
)
from tornado.ioloop import IOLoop, PeriodicCallback
import dask
from dask.highlevelgraph import HighLevelGraph
from dask.utils import format_bytes, format_time, parse_bytes, parse_timedelta, tmpfile
from dask.widgets import get_template
from distributed import cluster_dump, preloading, profile
from distributed import versions as version_module
from distributed._stories import scheduler_story
from distributed.active_memory_manager import ActiveMemoryManagerExtension, RetireWorker
from distributed.batched import BatchedSend
from distributed.comm import (
Comm,
CommClosedError,
get_address_host,
normalize_address,
resolve_address,
unparse_host_port,
)
from distributed.comm.addressing import addresses_from_user_args
from distributed.core import Status, clean_exception, rpc, send_recv
from distributed.diagnostics.memory_sampler import MemorySamplerExtension
from distributed.diagnostics.plugin import SchedulerPlugin, _get_plugin_name
from distributed.event import EventExtension
from distributed.http import get_handlers
from distributed.lock import LockExtension
from distributed.metrics import time
from distributed.multi_lock import MultiLockExtension
from distributed.node import ServerNode
from distributed.proctitle import setproctitle
from distributed.protocol.pickle import dumps, loads
from distributed.publish import PublishExtension
from distributed.pubsub import PubSubSchedulerExtension
from distributed.queues import QueueExtension
from distributed.recreate_tasks import ReplayTaskScheduler
from distributed.security import Security
from distributed.semaphore import SemaphoreExtension
from distributed.shuffle import ShuffleSchedulerExtension
from distributed.stealing import WorkStealing
from distributed.utils import (
All,
TimeoutError,
empty_context,
get_fileno_limit,
key_split,
key_split_group,
log_errors,
no_default,
recursive_to_dict,
validate_key,
)
from distributed.utils_comm import (
gather_from_workers,
retry_operation,
scatter_to_workers,
)
from distributed.utils_perf import disable_gc_diagnosis, enable_gc_diagnosis
from distributed.variable import VariableExtension
logger = logging.getLogger(__name__)
LOG_PDB = dask.config.get("distributed.admin.pdb-on-err")
DEFAULT_DATA_SIZE = parse_bytes(
dask.config.get("distributed.scheduler.default-data-size")
)
STIMULUS_ID_UNSET = "<stimulus_id unset>"
DEFAULT_EXTENSIONS = {
"locks": LockExtension,
"multi_locks": MultiLockExtension,
"publish": PublishExtension,
"replay-tasks": ReplayTaskScheduler,
"queues": QueueExtension,
"variables": VariableExtension,
"pubsub": PubSubSchedulerExtension,
"semaphores": SemaphoreExtension,
"events": EventExtension,
"amm": ActiveMemoryManagerExtension,
"memory_sampler": MemorySamplerExtension,
"shuffle": ShuffleSchedulerExtension,
"stealing": WorkStealing,
}
ALL_TASK_STATES = {"released", "waiting", "no-worker", "processing", "erred", "memory"}
[docs]class ClientState:
"""A simple object holding information about a client."""
#: A unique identifier for this client. This is generally an opaque
#: string generated by the client itself.
client_key: str
#: Cached hash of :attr:`~ClientState.client_key`
_hash: int
#: A set of tasks this client wants to be kept in memory, so that it can download
#: its result when desired. This is the reverse mapping of
#: :class:`TaskState.who_wants`. Tasks are typically removed from this set when the
#: corresponding object in the client's space (for example a ``Future`` or a Dask
#: collection) gets garbage-collected.
wants_what: set[TaskState]
#: The last time we received a heartbeat from this client, in local scheduler time.
last_seen: float
#: Output of :func:`distributed.versions.get_versions` on the client
versions: dict[str, Any]
__slots__ = tuple(__annotations__)
def __init__(self, client: str, *, versions: dict[str, Any] | None = None):
self.client_key = client
self._hash = hash(client)
self.wants_what = set()
self.last_seen = time()
self.versions = versions or {}
def __hash__(self) -> int:
return self._hash
def __eq__(self, other: object) -> bool:
if not isinstance(other, ClientState):
return False
return self.client_key == other.client_key
def __repr__(self) -> str:
return f"<Client {self.client_key!r}>"
def __str__(self) -> str:
return self.client_key
def _to_dict_no_nest(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
TaskState._to_dict
"""
return recursive_to_dict(
self,
exclude=set(exclude) | {"versions"}, # type: ignore
members=True,
)
[docs]class MemoryState:
"""Memory readings on a worker or on the whole cluster.
See :doc:`worker-memory`.
Attributes / properties:
managed
Sum of the output of sizeof() for all dask keys held by the worker in memory,
plus number of bytes spilled to disk
managed_in_memory
Sum of the output of sizeof() for the dask keys held in RAM. Note that this may
be inaccurate, which may cause inaccurate unmanaged memory (see below).
managed_spilled
Number of bytes for the dask keys spilled to the hard drive.
Note that this is the size on disk; size in memory may be different due to
compression and inaccuracies in sizeof(). In other words, given the same keys,
'managed' will change depending if the keys are in memory or spilled.
process
Total RSS memory measured by the OS on the worker process.
This is always exactly equal to managed_in_memory + unmanaged.
unmanaged
process - managed_in_memory. This is the sum of
- Python interpreter and modules
- global variables
- memory temporarily allocated by the dask tasks that are currently running
- memory fragmentation
- memory leaks
- memory not yet garbage collected
- memory not yet free()'d by the Python memory manager to the OS
unmanaged_old
Minimum of the 'unmanaged' measures over the last
``distributed.memory.recent-to-old-time`` seconds
unmanaged_recent
unmanaged - unmanaged_old; in other words process memory that has been recently
allocated but is not accounted for by dask; hopefully it's mostly a temporary
spike.
optimistic
managed_in_memory + unmanaged_old; in other words the memory held long-term by
the process under the hopeful assumption that all unmanaged_recent memory is a
temporary spike
.. note::
There is an intentional misalignment in terminology between this class (which is
meant for internal / programmatic use) and the memory readings on the GUI (which
is aimed at the general public:
================= =====================
MemoryState GUI
================= =====================
managed n/a
managed_in_memory managed
managed_spilled spilled
process process (RSS); memory
unmanaged n/a
unmanaged_old unmanaged (old)
unmanaged_recent unmanaged (recent)
optimistic n/a
================= =====================
"""
process: int
unmanaged_old: int
managed_in_memory: int
managed_spilled: int
__slots__ = tuple(__annotations__)
def __init__(
self,
*,
process: int,
unmanaged_old: int,
managed_in_memory: int,
managed_spilled: int,
):
# Some data arrives with the heartbeat, some other arrives in realtime as the
# tasks progress. Also, sizeof() is not guaranteed to return correct results.
# This can cause glitches where a partial measure is larger than the whole, so
# we need to force all numbers to add up exactly by definition.
self.process = process
self.managed_in_memory = min(self.process, managed_in_memory)
self.managed_spilled = managed_spilled
# Subtractions between unsigned ints guaranteed by construction to be >= 0
self.unmanaged_old = min(unmanaged_old, process - self.managed_in_memory)
@staticmethod
def sum(*infos: MemoryState) -> MemoryState:
process = 0
unmanaged_old = 0
managed_in_memory = 0
managed_spilled = 0
for ms in infos:
process += ms.process
unmanaged_old += ms.unmanaged_old
managed_spilled += ms.managed_spilled
managed_in_memory += ms.managed_in_memory
return MemoryState(
process=process,
unmanaged_old=unmanaged_old,
managed_in_memory=managed_in_memory,
managed_spilled=managed_spilled,
)
@property
def managed(self) -> int:
return self.managed_in_memory + self.managed_spilled
@property
def unmanaged(self) -> int:
# This is never negative thanks to __init__
return self.process - self.managed_in_memory
@property
def unmanaged_recent(self) -> int:
# This is never negative thanks to __init__
return self.process - self.managed_in_memory - self.unmanaged_old
@property
def optimistic(self) -> int:
return self.managed_in_memory + self.unmanaged_old
def __repr__(self) -> str:
return (
f"Process memory (RSS) : {format_bytes(self.process)}\n"
f" - managed by Dask : {format_bytes(self.managed_in_memory)}\n"
f" - unmanaged (old) : {format_bytes(self.unmanaged_old)}\n"
f" - unmanaged (recent): {format_bytes(self.unmanaged_recent)}\n"
f"Spilled to disk : {format_bytes(self.managed_spilled)}\n"
)
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
"""
return recursive_to_dict(self, exclude=exclude, members=True)
[docs]class WorkerState:
"""A simple object holding information about a worker.
Not to be confused with :class:`distributed.worker_state_machine.WorkerState`.
"""
#: This worker's unique key. This can be its connected address
#: (such as ``"tcp://127.0.0.1:8891"``) or an alias (such as ``"alice"``).
address: str
pid: int
name: object
#: The number of CPU threads made available on this worker
nthreads: int
#: Memory available to the worker, in bytes
memory_limit: int
local_directory: str
services: dict[str, int]
#: Output of :meth:`distributed.versions.get_versions` on the worker
versions: dict[str, Any]
#: Address of the associated :class:`~distributed.nanny.Nanny`, if present
nanny: str
#: Read-only worker status, synced one way from the remote Worker object
status: Status
#: Cached hash of :attr:`~WorkerState.address`
_hash: int
#: The total memory size, in bytes, used by the tasks this worker holds in memory
#: (i.e. the tasks in this worker's :attr:`~WorkerState.has_what`).
nbytes: int
#: The total expected runtime, in seconds, of all tasks currently processing on this
#: worker. This is the sum of all the costs in this worker's
# :attr:`~WorkerState.processing` dictionary.
occupancy: float
#: Worker memory unknown to the worker, in bytes, which has been there for more than
#: 30 seconds. See :class:`MemoryState`.
_memory_unmanaged_old: int
#: History of the last 30 seconds' worth of unmanaged memory. Used to differentiate
#: between "old" and "new" unmanaged memory.
#: Format: ``[(timestamp, bytes), (timestamp, bytes), ...]``
_memory_unmanaged_history: deque[tuple[float, int]]
metrics: dict[str, Any]
#: The last time we received a heartbeat from this worker, in local scheduler time.
last_seen: float
time_delay: float
bandwidth: float
#: A set of all TaskStates on this worker that are actors. This only includes those
#: actors whose state actually lives on this worker, not actors to which this worker
#: has a reference.
actors: set[TaskState]
#: Underlying data of :meth:`WorkerState.has_what`
_has_what: dict[TaskState, None]
#: A dictionary of tasks that have been submitted to this worker. Each task state is
#: associated with the expected cost in seconds of running that task, summing both
#: the task's expected computation time and the expected communication time of its
#: result.
#:
#: If a task is already executing on the worker and the excecution time is twice the
#: learned average TaskGroup duration, this will be set to twice the current
#: executing time. If the task is unknown, the default task duration is used instead
#: of the TaskGroup average.
#:
#: Multiple tasks may be submitted to a worker in advance and the worker will run
#: them eventually, depending on its execution resources (but see
#: :doc:`work-stealing`).
#:
#: All the tasks here are in the "processing" state.
#: This attribute is kept in sync with :attr:`TaskState.processing_on`.
processing: dict[TaskState, float]
#: Running tasks that invoked :func:`distributed.secede`
long_running: set[TaskState]
#: A dictionary of tasks that are currently being run on this worker.
#: Each task state is asssociated with the duration in seconds which the task has
#: been running.
executing: dict[TaskState, float]
#: The available resources on this worker, e.g. ``{"GPU": 2}``.
#: These are abstract quantities that constrain certain tasks from running at the
#: same time on this worker.
resources: dict[str, float]
#: The sum of each resource used by all tasks allocated to this worker.
#: The numbers in this dictionary can only be less or equal than those in this
#: worker's :attr:`~WorkerState.resources`.
used_resources: dict[str, float]
#: Arbitrary additional metadata to be added to :meth:`~WorkerState.identity`
extra: dict[str, Any]
# The unique server ID this WorkerState is referencing
server_id: str
__slots__ = tuple(__annotations__)
def __init__(
self,
*,
address: str,
status: Status,
pid: int,
name: object,
nthreads: int = 0,
memory_limit: int,
local_directory: str,
nanny: str,
server_id: str,
services: dict[str, int] | None = None,
versions: dict[str, Any] | None = None,
extra: dict[str, Any] | None = None,
):
self.server_id = server_id
self.address = address
self.pid = pid
self.name = name
self.nthreads = nthreads
self.memory_limit = memory_limit
self.local_directory = local_directory
self.services = services or {}
self.versions = versions or {}
self.nanny = nanny
self.status = status
self._hash = hash(self.server_id)
self.nbytes = 0
self.occupancy = 0
self._memory_unmanaged_old = 0
self._memory_unmanaged_history = deque()
self.metrics = {}
self.last_seen = 0
self.time_delay = 0
self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))
self.actors = set()
self._has_what = {}
self.processing = {}
self.long_running = set()
self.executing = {}
self.resources = {}
self.used_resources = {}
self.extra = extra or {}
def __hash__(self) -> int:
return self._hash
def __eq__(self, other: object) -> bool:
return isinstance(other, WorkerState) and other.server_id == self.server_id
@property
def has_what(self) -> Set[TaskState]:
"""An insertion-sorted set-like of tasks which currently reside on this worker.
All the tasks here are in the "memory" state.
This is the reverse mapping of :attr:`TaskState.who_has`.
This is a read-only public accessor. The data is implemented as a dict without
values, because rebalance() relies on dicts being insertion-sorted.
"""
return self._has_what.keys()
@property
def host(self) -> str:
return get_address_host(self.address)
@property
def memory(self) -> MemoryState:
return MemoryState(
# metrics["memory"] is None if the worker sent a heartbeat before its
# SystemMonitor ever had a chance to run
process=self.metrics["memory"] or 0,
# self.nbytes is instantaneous; metrics may lag behind by a heartbeat
managed_in_memory=max(
0, self.nbytes - self.metrics["spilled_nbytes"]["memory"]
),
managed_spilled=self.metrics["spilled_nbytes"]["disk"],
unmanaged_old=self._memory_unmanaged_old,
)
[docs] def clean(self) -> WorkerState:
"""Return a version of this object that is appropriate for serialization"""
ws = WorkerState(
address=self.address,
status=self.status,
pid=self.pid,
name=self.name,
nthreads=self.nthreads,
memory_limit=self.memory_limit,
local_directory=self.local_directory,
services=self.services,
nanny=self.nanny,
extra=self.extra,
server_id=self.server_id,
)
ws.processing = {
ts.key: cost for ts, cost in self.processing.items() # type: ignore
}
ws.executing = {
ts.key: duration for ts, duration in self.executing.items() # type: ignore
}
return ws
def __repr__(self) -> str:
name = f", name: {self.name}" if self.name != self.address else ""
return (
f"<WorkerState {self.address!r}{name}, "
f"status: {self.status.name}, "
f"memory: {len(self.has_what)}, "
f"processing: {len(self.processing)}>"
)
def _repr_html_(self):
return get_template("worker_state.html.j2").render(
address=self.address,
name=self.name,
status=self.status.name,
has_what=self.has_what,
processing=self.processing,
)
def identity(self) -> dict[str, Any]:
return {
"type": "Worker",
"id": self.name,
"host": self.host,
"resources": self.resources,
"local_directory": self.local_directory,
"name": self.name,
"nthreads": self.nthreads,
"memory_limit": self.memory_limit,
"last_seen": self.last_seen,
"services": self.services,
"metrics": self.metrics,
"status": self.status.name,
"nanny": self.nanny,
**self.extra,
}
def _to_dict_no_nest(self, *, exclude: Container[str] = ()) -> dict[str, Any]:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
TaskState._to_dict
"""
return recursive_to_dict(
self,
exclude=set(exclude) | {"versions"}, # type: ignore
members=True,
)
class Computation:
"""Collection tracking a single compute or persist call
See also
--------
TaskPrefix
TaskGroup
TaskState
"""
start: float
groups: set[TaskGroup]
code: SortedSet
id: uuid.UUID
__slots__ = tuple(__annotations__)
def __init__(self):
self.start = time()
self.groups = set()
self.code = SortedSet()
self.id = uuid.uuid4()
@property
def stop(self) -> float:
if self.groups:
return max(tg.stop for tg in self.groups)
else:
return -1
@property
def states(self) -> dict[str, int]:
return merge_with(sum, (tg.states for tg in self.groups))
def __repr__(self) -> str:
return (
f"<Computation {self.id}: "
+ "Tasks: "
+ ", ".join(
"%s: %d" % (k, v) for (k, v) in sorted(self.states.items()) if v
)
+ ">"
)
def _repr_html_(self):
return get_template("computation.html.j2").render(
id=self.id,
start=self.start,
stop=self.stop,
groups=self.groups,
states=self.states,
code=self.code,
)
class TaskPrefix:
"""Collection tracking all tasks within a group
Keys often have a structure like ``("x-123", 0)``
A group takes the first section, like ``"x"``
See Also
--------
TaskGroup
"""
#: The name of a group of tasks.
#: For a task like ``("x-123", 0)`` this is the text ``"x"``
name: str
#: An exponentially weighted moving average duration of all tasks with this prefix
duration_average: float
#: Numbers of times a task was marked as suspicious with this prefix
suspicious: int
#: Store timings for each prefix-action
all_durations: defaultdict[str, float]
#: Task groups associated to this prefix
groups: list[TaskGroup]
__slots__ = tuple(__annotations__)
def __init__(self, name: str):
self.name = name
self.groups = []
self.all_durations = defaultdict(float)
task_durations = dask.config.get("distributed.scheduler.default-task-durations")
if self.name in task_durations:
self.duration_average = parse_timedelta(task_durations[self.name])
else:
self.duration_average = -1
self.suspicious = 0
def add_duration(self, action: str, start: float, stop: float) -> None:
duration = stop - start
self.all_durations[action] += duration
if action == "compute":
old = self.duration_average
if old < 0:
self.duration_average = duration
else:
self.duration_average = 0.5 * duration + 0.5 * old
@property
def states(self) -> dict[str, int]:
"""The number of tasks in each state,
like ``{"memory": 10, "processing": 3, "released": 4, ...}``
"""
return merge_with(sum, [tg.states for tg in self.groups])
@property
def active(self) -> list[TaskGroup]:
return [
tg
for tg in self.groups
if any(k != "forgotten" and v != 0 for k, v in tg.states.items())
]
@property
def active_states(self) -> dict[str, int]:
return merge_with(sum, [tg.states for tg in self.active])
def __repr__(self) -> str:
return (
"<"
+ self.name
+ ": "
+ ", ".join(
"%s: %d" % (k, v) for (k, v) in sorted(self.states.items()) if v
)
+ ">"
)
@property
def nbytes_total(self) -> int:
return sum(tg.nbytes_total for tg in self.groups)
def __len__(self) -> int:
return sum(map(len, self.groups))
@property
def duration(self) -> float:
return sum(tg.duration for tg in self.groups)
@property
def types(self) -> set[str]:
return {typ for tg in self.groups for typ in tg.types}
class TaskGroup:
"""Collection tracking all tasks within a group
Keys often have a structure like ``("x-123", 0)``
A group takes the first section, like ``"x-123"``
See also
--------
TaskPrefix
"""
#: The name of a group of tasks.
#: For a task like ``("x-123", 0)`` this is the text ``"x-123"``
name: str
#: The number of tasks in each state,
#: like ``{"memory": 10, "processing": 3, "released": 4, ...}``
states: dict[str, int]
#: The other TaskGroups on which this one depends
dependencies: set[TaskGroup]
#: The total number of bytes that this task group has produced
nbytes_total: int
#: The total amount of time spent on all tasks in this TaskGroup
duration: float
#: The result types of this TaskGroup
types: set[str]
#: The worker most recently assigned a task from this group, or None when the group
#: is not identified to be root-like by `SchedulerState.decide_worker`.
last_worker: WorkerState | None
#: If `last_worker` is not None, the number of times that worker should be assigned
#: subsequent tasks until a new worker is chosen.
last_worker_tasks_left: int
prefix: TaskPrefix | None
start: float
stop: float
all_durations: defaultdict[str, float]
__slots__ = tuple(__annotations__)
def __init__(self, name: str):
self.name = name
self.prefix = None
self.states = {state: 0 for state in ALL_TASK_STATES}
self.states["forgotten"] = 0
self.dependencies = set()
self.nbytes_total = 0
self.duration = 0
self.types = set()
self.start = 0.0
self.stop = 0.0
self.all_durations = defaultdict(float)
self.last_worker = None
self.last_worker_tasks_left = 0
def add_duration(self, action: str, start: float, stop: float) -> None:
duration = stop - start
self.all_durations[action] += duration
if action == "compute":
if self.stop < stop:
self.stop = stop
self.start = self.start or start
self.duration += duration
assert self.prefix is not None
self.prefix.add_duration(action, start, stop)
def add(self, other: TaskState) -> None:
self.states[other.state] += 1
other.group = self
def __repr__(self) -> str:
return (
"<"
+ (self.name or "no-group")
+ ": "
+ ", ".join(
"%s: %d" % (k, v) for (k, v) in sorted(self.states.items()) if v
)
+ ">"
)
def __len__(self) -> int:
return sum(self.states.values())
def _to_dict_no_nest(self, *, exclude: Container[str] = ()) -> dict[str, Any]:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
TaskState._to_dict
"""
return recursive_to_dict(self, exclude=exclude, members=True)
[docs]class TaskState:
"""A simple object holding information about a task.
Not to be confused with :class:`distributed.worker_state_machine.TaskState`, which
holds similar information on the Worker side.
"""
#: The key is the unique identifier of a task, generally formed from the name of the
#: function, followed by a hash of the function and arguments, like
#: ``'inc-ab31c010444977004d656610d2d421ec'``.
key: str
#: The broad class of tasks to which this task belongs like "inc" or "read_csv"
prefix: TaskPrefix
#: A specification of how to run the task. The type and meaning of this value is
#: opaque to the scheduler, as it is only interpreted by the worker to which the
#: task is sent for executing.
#:
#: As a special case, this attribute may also be ``None``, in which case the task is
#: "pure data" (such as, for example, a piece of data loaded in the scheduler using
#: :meth:`Client.scatter`). A "pure data" task cannot be computed again if its
#: value is lost.
run_spec: object
#: The priority provides each task with a relative ranking which is used to break
#: ties when many tasks are being considered for execution.
#:
#: This ranking is generally a 2-item tuple. The first (and dominant) item
#: corresponds to when it was submitted. Generally, earlier tasks take precedence.
#: The second item is determined by the client, and is a way to prioritize tasks
#: within a large graph that may be important, such as if they are on the critical
#: path, or good to run in order to release many dependencies. This is explained
#: further in :doc:`Scheduling Policy <scheduling-policies>`.
priority: tuple[int, ...]
# Attribute underlying the state property
_state: str
#: The set of tasks this task depends on for proper execution. Only tasks still
#: alive are listed in this set. If, for whatever reason, this task also depends on
#: a forgotten task, the :attr:`has_lost_dependencies` flag is set.
#:
#: A task can only be executed once all its dependencies have already been
#: successfully executed and have their result stored on at least one worker. This
#: is tracked by progressively draining the :attr:`waiting_on` set.
dependencies: set[TaskState]
#: The set of tasks which depend on this task. Only tasks still alive are listed in
#: this set. This is the reverse mapping of :attr:`dependencies`.
dependents: set[TaskState]
#: Whether any of the dependencies of this task has been forgotten. For memory
#: consumption reasons, forgotten tasks are not kept in memory even though they may
#: have dependent tasks. When a task is forgotten, therefore, each of its
#: dependents has their :attr:`has_lost_dependencies` attribute set to ``True``.
#:
#: If :attr:`has_lost_dependencies` is true, this task cannot go into the
#: "processing" state anymore.
has_lost_dependencies: bool
#: The set of tasks this task is waiting on *before* it can be executed. This is
#: always a subset of :attr:`dependencies`. Each time one of the dependencies has
#: finished processing, it is removed from the :attr:`waiting_on` set.
#:
#: Once :attr:`waiting_on` becomes empty, this task can move from the "waiting"
#: state to the "processing" state (unless one of the dependencies errored out, in
#: which case this task is instead marked "erred").
waiting_on: set[TaskState]
#: The set of tasks which need this task to remain alive. This is always a subset
#: of :attr:`dependents`. Each time one of the dependents has finished processing,
#: it is removed from the :attr:`waiters` set.
#:
#: Once both :attr:`waiters` and :attr:`who_wants` become empty, this task can be
#: released (if it has a non-empty :attr:`run_spec`) or forgotten (otherwise) by the
#: scheduler, and by any workers in :attr:`who_has`.
#:
#: .. note::
#: Counter-intuitively, :attr:`waiting_on` and :attr:`waiters` are not reverse
#: mappings of each other.
waiters: set[TaskState]
#: The set of clients who want the result of this task to remain alive.
#: This is the reverse mapping of :attr:`ClientState.wants_what`.
#:
#: When a client submits a graph to the scheduler it also specifies which output
#: tasks it desires, such that their results are not released from memory.
#:
#: Once a task has finished executing (i.e. moves into the "memory" or "erred"
#: state), the clients in :attr:`who_wants` are notified.
#:
#: Once both :attr:`waiters` and :attr:`who_wants` become empty, this task can be
#: released (if it has a non-empty :attr:`run_spec`) or forgotten (otherwise) by the
#: scheduler, and by any workers in :attr:`who_has`.
who_wants: set[ClientState]
#: The set of workers who have this task's result in memory. It is non-empty iff the
#: task is in the "memory" state. There can be more than one worker in this set if,
#: for example, :meth:`Client.scatter` or :meth:`Client.replicate` was used.
#:
#: This is the reverse mapping of :attr:`WorkerState.has_what`.
who_has: set[WorkerState]
#: If this task is in the "processing" state, which worker is currently processing
#: it. This attribute is kept in sync with :attr:`WorkerState.processing`.
processing_on: WorkerState | None
#: The number of times this task can automatically be retried in case of failure.
#: If a task fails executing (the worker returns with an error), its :attr:`retries`
#: attribute is checked. If it is equal to 0, the task is marked "erred". If it is
#: greater than 0, the :attr:`retries` attribute is decremented and execution is
#: attempted again.
retries: int
#: The number of bytes, as determined by ``sizeof``, of the result of a finished
#: task. This number is used for diagnostics and to help prioritize work.
#: Set to -1 for unfinished tasks.
nbytes: int
#: The type of the object as a string. Only present for tasks that have been
#: computed.
type: str
#: If this task failed executing, the exception object is stored here.
exception: object | None
#: If this task failed executing, the traceback object is stored here.
traceback: object | None
#: string representation of exception
exception_text: str
#: string representation of traceback
traceback_text: str
#: If this task or one of its dependencies failed executing, the failed task is
#: stored here (possibly itself).
exception_blame: TaskState | None
#: Worker addresses on which errors appeared, causing this task to be in an error
#: state.
erred_on: set[str]
#: The number of times this task has been involved in a worker death.
#:
#: Some tasks may cause workers to die (such as calling ``os._exit(0)``). When a
#: worker dies, all of the tasks on that worker are reassigned to others. This
#: combination of behaviors can cause a bad task to catastrophically destroy all
#: workers on the cluster, one after another. Whenever a worker dies, we mark each
#: task currently processing on that worker (as recorded by
#: :attr:`WorkerState.processing`) as suspicious. If a task is involved in three
#: deaths (or some other fixed constant) then we mark the task as ``erred``.
suspicious: int
#: A set of hostnames where this task can be run (or ``None`` if empty). Usually
#: this is empty unless the task has been specifically restricted to only run on
#: certain hosts. A hostname may correspond to one or several connected workers.
host_restrictions: set[str]
#: A set of complete worker addresses where this can be run (or ``None`` if empty).
#: Usually this is empty unless the task has been specifically restricted to only
#: run on certain workers.
#: Note this is tracking worker addresses, not worker states, since the specific
#: workers may not be connected at this time.
worker_restrictions: set[str]
#: Resources required by this task, such as ``{'gpu': 1}`` or ``{'memory': 1e9}``
#: These are user-defined names and are matched against the : contents of each
#: :attr:`WorkerState.resources` dictionary.
resource_restrictions: dict[str, float]
#: False
#: Each of :attr:`host_restrictions`, :attr:`worker_restrictions` and
#: :attr:`resource_restrictions` is a hard constraint: if no worker is available
#: satisfying those restrictions, the task cannot go into the "processing" state
#: and will instead go into the "no-worker" state.
#: True
#: The above restrictions are mere preferences: if no worker is available
#: satisfying those restrictions, the task can still go into the
#: "processing" state and be sent for execution to another connected worker.
loose_restrictions: bool
#: Whether or not this task is an Actor
actor: bool
#: The group of tasks to which this one belongs
group: TaskGroup
#: Same as of group.name
group_key: str
#: Metadata related to task
metadata: dict[str, Any]
#: Task annotations
annotations: dict[str, Any]
#: Cached hash of :attr:`~TaskState.client_key`
_hash: int
# Support for weakrefs to a class with __slots__
__weakref__: Any = None
__slots__ = tuple(__annotations__)
# Instances not part of slots since class variable
_instances: ClassVar[weakref.WeakSet[TaskState]] = weakref.WeakSet()
def __init__(self, key: str, run_spec: object):
self.key = key
self._hash = hash(key)
self.run_spec = run_spec
self._state = None # type: ignore
self.exception = None
self.exception_blame = None
self.traceback = None
self.exception_text = ""
self.traceback_text = ""
self.suspicious = 0
self.retries = 0
self.nbytes = -1
self.priority = None # type: ignore
self.who_wants = set()
self.dependencies = set()
self.dependents = set()
self.waiting_on = set()
self.waiters = set()
self.who_has = set()
self.processing_on = None
self.has_lost_dependencies = False
self.host_restrictions = None # type: ignore
self.worker_restrictions = None # type: ignore
self.resource_restrictions = {}
self.loose_restrictions = False
self.actor = False
self.prefix = None # type: ignore
self.type = None # type: ignore
self.group_key = key_split_group(key)
self.group = None # type: ignore
self.metadata = {}
self.annotations = {}
self.erred_on = set()
TaskState._instances.add(self)
def __hash__(self) -> int:
return self._hash
def __eq__(self, other: object) -> bool:
return isinstance(other, TaskState) and self.key == other.key
@property
def state(self) -> str:
"""This task's current state. Valid states include ``released``, ``waiting``,
``no-worker``, ``processing``, ``memory``, ``erred`` and ``forgotten``. If it
is ``forgotten``, the task isn't stored in the ``tasks`` dictionary anymore and
will probably disappear soon from memory.
"""
return self._state
@state.setter
def state(self, value: str) -> None:
self.group.states[self._state] -= 1
self.group.states[value] += 1
self._state = value
[docs] def add_dependency(self, other: TaskState) -> None:
"""Add another task as a dependency of this task"""
self.dependencies.add(other)
self.group.dependencies.add(other.group)
other.dependents.add(self)
def get_nbytes(self) -> int:
return self.nbytes if self.nbytes >= 0 else DEFAULT_DATA_SIZE
def set_nbytes(self, nbytes: int) -> None:
diff = nbytes
old_nbytes = self.nbytes
if old_nbytes >= 0:
diff -= old_nbytes
self.group.nbytes_total += diff
for ws in self.who_has:
ws.nbytes += diff
self.nbytes = nbytes
def __repr__(self) -> str:
return f"<TaskState {self.key!r} {self._state}>"
def _repr_html_(self):
return get_template("task_state.html.j2").render(
state=self.state,
nbytes=self.nbytes,
key=self.key,
)
def validate(self) -> None:
try:
for cs in self.who_wants:
assert isinstance(cs, ClientState), (repr(cs), self.who_wants)
for ws in self.who_has:
assert isinstance(ws, WorkerState), (repr(ws), self.who_has)
for ts in self.dependencies:
assert isinstance(ts, TaskState), (repr(ts), self.dependencies)
for ts in self.dependents:
assert isinstance(ts, TaskState), (repr(ts), self.dependents)
validate_task_state(self)
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
def get_nbytes_deps(self):
return sum(ts.get_nbytes() for ts in self.dependencies)
def _to_dict_no_nest(self, *, exclude: Container[str] = ()) -> dict[str, Any]:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
Notes
-----
This class uses ``_to_dict_no_nest`` instead of ``_to_dict``.
When a task references another task, or when a WorkerState.tasks contains tasks,
this method is not executed for the inner task, even if the inner task was never
seen before; you get a repr instead. All tasks should neatly appear under
Scheduler.tasks. This also prevents a RecursionError during particularly heavy
loads, which have been observed to happen whenever there's an acyclic dependency
chain of ~200+ tasks.
"""
return recursive_to_dict(self, exclude=exclude, members=True)
class SchedulerState:
"""Underlying task state of dynamic scheduler
Tracks the current state of workers, data, and computations.
Handles transitions between different task states. Notifies the
Scheduler of changes by messaging passing through Queues, which the
Scheduler listens to responds accordingly.
All events are handled quickly, in linear time with respect to their
input (which is often of constant size) and generally within a
millisecond.
Users typically do not interact with ``Transitions`` directly. Instead
users interact with the ``Client``, which in turn engages the
``Scheduler`` affecting different transitions here under-the-hood. In
the background ``Worker``s also engage with the ``Scheduler``
affecting these state transitions as well.
**State**
The ``Transitions`` object contains the following state variables.
Each variable is listed along with what it stores and a brief
description.
* **tasks:** ``{task key: TaskState}``
Tasks currently known to the scheduler
* **unrunnable:** ``{TaskState}``
Tasks in the "no-worker" state
* **workers:** ``{worker key: WorkerState}``
Workers currently connected to the scheduler
* **idle:** ``{WorkerState}``:
Set of workers that are not fully utilized
* **saturated:** ``{WorkerState}``:
Set of workers that are not over-utilized
* **running:** ``{WorkerState}``:
Set of workers that are currently in running state
* **clients:** ``{client key: ClientState}``
Clients currently connected to the scheduler
* **task_duration:** ``{key-prefix: time}``
Time we expect certain functions to take, e.g. ``{'sum': 0.25}``
"""
__slots__ = {
"aliases",
"bandwidth",
"clients",
"computations",
"extensions",
"host_info",
"idle",
"n_tasks",
"resources",
"saturated",
"running",
"tasks",
"task_groups",
"task_prefixes",
"task_metadata",
"replicated_tasks",
"total_nthreads",
"total_occupancy",
"unknown_durations",
"unrunnable",
"validate",
"workers",
"transition_counter",
"_idle_transition_counter",
"transition_counter_max",
"plugins",
"UNKNOWN_TASK_DURATION",
"MEMORY_RECENT_TO_OLD_TIME",
"MEMORY_REBALANCE_MEASURE",
"MEMORY_REBALANCE_SENDER_MIN",
"MEMORY_REBALANCE_RECIPIENT_MAX",
"MEMORY_REBALANCE_HALF_GAP",
}
def __init__(
self,
aliases: dict,
clients: dict[str, ClientState],
workers: SortedDict[str, WorkerState],
host_info: dict,
resources: dict,
tasks: dict,
unrunnable: set,
validate: bool,
plugins: Iterable[SchedulerPlugin] = (),
transition_counter_max: int | Literal[False] = False,
**kwargs, # Passed verbatim to Server.__init__()
):
logger.info("State start")
self.aliases = aliases
self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))
self.clients = clients
self.clients["fire-and-forget"] = ClientState("fire-and-forget")
self.extensions = {} # type: ignore
self.host_info = host_info
self.idle = SortedDict()
self.n_tasks = 0
self.resources = resources
self.saturated: set[WorkerState] = set()
self.tasks = tasks
self.replicated_tasks = {
ts for ts in self.tasks.values() if len(ts.who_has) > 1
}
self.computations: deque[Computation] = deque(
maxlen=dask.config.get("distributed.diagnostics.computations.max-history")
)
self.task_groups: dict[str, TaskGroup] = {}
self.task_prefixes: dict[str, TaskPrefix] = {}
self.task_metadata = {} # type: ignore
self.total_nthreads = 0
self.total_occupancy = 0.0
self.unknown_durations: dict[str, set[TaskState]] = {}
self.unrunnable = unrunnable
self.validate = validate
self.workers = workers
self.running = {
ws for ws in self.workers.values() if ws.status == Status.running
}
self.plugins = {} if not plugins else {_get_plugin_name(p): p for p in plugins}
# Variables from dask.config, cached by __init__ for performance
self.UNKNOWN_TASK_DURATION = parse_timedelta(
dask.config.get("distributed.scheduler.unknown-task-duration")
)
self.MEMORY_RECENT_TO_OLD_TIME = parse_timedelta(
dask.config.get("distributed.worker.memory.recent-to-old-time")
)
self.MEMORY_REBALANCE_MEASURE = dask.config.get(
"distributed.worker.memory.rebalance.measure"
)
self.MEMORY_REBALANCE_SENDER_MIN = dask.config.get(
"distributed.worker.memory.rebalance.sender-min"
)
self.MEMORY_REBALANCE_RECIPIENT_MAX = dask.config.get(
"distributed.worker.memory.rebalance.recipient-max"
)
self.MEMORY_REBALANCE_HALF_GAP = (
dask.config.get("distributed.worker.memory.rebalance.sender-recipient-gap")
/ 2.0
)
self.transition_counter = 0
self._idle_transition_counter = 0
self.transition_counter_max = transition_counter_max
@property
def memory(self) -> MemoryState:
return MemoryState.sum(*(w.memory for w in self.workers.values()))
@property
def __pdict__(self):
return {
"bandwidth": self.bandwidth,
"resources": self.resources,
"saturated": self.saturated,
"unrunnable": self.unrunnable,
"n_tasks": self.n_tasks,
"unknown_durations": self.unknown_durations,
"validate": self.validate,
"tasks": self.tasks,
"task_groups": self.task_groups,
"task_prefixes": self.task_prefixes,
"total_nthreads": self.total_nthreads,
"total_occupancy": self.total_occupancy,
"extensions": self.extensions,
"clients": self.clients,
"workers": self.workers,
"idle": self.idle,
"host_info": self.host_info,
}
def new_task(
self, key: str, spec: object, state: str, computation: Computation | None = None
) -> TaskState:
"""Create a new task, and associated states"""
ts = TaskState(key, spec)
ts._state = state
prefix_key = key_split(key)
tp = self.task_prefixes.get(prefix_key)
if tp is None:
self.task_prefixes[prefix_key] = tp = TaskPrefix(prefix_key)
ts.prefix = tp
group_key = ts.group_key
tg = self.task_groups.get(group_key)
if tg is None:
self.task_groups[group_key] = tg = TaskGroup(group_key)
if computation:
computation.groups.add(tg)
tg.prefix = tp
tp.groups.append(tg)
tg.add(ts)
self.tasks[key] = ts
return ts
#####################
# State Transitions #
#####################
def _transition(
self, key: str, finish: str, stimulus_id: str, *args, **kwargs
) -> tuple[dict, dict, dict]:
"""Transition a key from its current state to the finish state
Examples
--------
>>> self._transition('x', 'waiting')
{'x': 'processing'}
Returns
-------
Dictionary of recommendations for future transitions
See Also
--------
Scheduler.transitions : transitive version of this function
"""
try:
ts = self.tasks.get(key)
if ts is None:
return {}, {}, {}
start = ts._state
if start == finish:
return {}, {}, {}
# Notes:
# - in case of transition through released, this counter is incremented by 2
# - this increase happens before the actual transitions, so that it can
# catch potential infinite recursions
self.transition_counter += 1
if self.transition_counter_max:
assert self.transition_counter < self.transition_counter_max
recommendations: dict = {}
worker_msgs: dict = {}
client_msgs: dict = {}
if self.plugins:
dependents = set(ts.dependents)
dependencies = set(ts.dependencies)
func = self._TRANSITIONS_TABLE.get((start, finish))
if func is not None:
recommendations, client_msgs, worker_msgs = func(
self, key, stimulus_id, *args, **kwargs
)
elif "released" not in (start, finish):
assert not args and not kwargs, (args, kwargs, start, finish)
a_recs, a_cmsgs, a_wmsgs = self._transition(
key, "released", stimulus_id
)
v = a_recs.get(key, finish)
func = self._TRANSITIONS_TABLE["released", v]
b_recs, b_cmsgs, b_wmsgs = func(self, key, stimulus_id)
recommendations.update(a_recs)
for c, new_msgs in a_cmsgs.items():
msgs = client_msgs.get(c)
if msgs is not None:
msgs.extend(new_msgs)
else:
client_msgs[c] = new_msgs
for w, new_msgs in a_wmsgs.items():
msgs = worker_msgs.get(w)
if msgs is not None:
msgs.extend(new_msgs)
else:
worker_msgs[w] = new_msgs
recommendations.update(b_recs)
for c, new_msgs in b_cmsgs.items():
msgs = client_msgs.get(c)
if msgs is not None:
msgs.extend(new_msgs)
else:
client_msgs[c] = new_msgs
for w, new_msgs in b_wmsgs.items():
msgs = worker_msgs.get(w)
if msgs is not None:
msgs.extend(new_msgs)
else:
worker_msgs[w] = new_msgs
start = "released"
else:
raise RuntimeError(f"Impossible transition from {start} to {finish}")
if not stimulus_id:
stimulus_id = STIMULUS_ID_UNSET
finish2 = ts._state
# FIXME downcast antipattern
scheduler = cast(Scheduler, self)
scheduler.transition_log.append(
(key, start, finish2, recommendations, stimulus_id, time())
)
if self.validate:
if stimulus_id == STIMULUS_ID_UNSET:
raise RuntimeError(
"stimulus_id not set during Scheduler transition"
)
logger.debug(
"Transitioned %r %s->%s (actual: %s). Consequence: %s",
key,
start,
finish2,
ts.state,
dict(recommendations),
)
if self.plugins:
# Temporarily put back forgotten key for plugin to retrieve it
if ts._state == "forgotten":
ts.dependents = dependents
ts.dependencies = dependencies
self.tasks[ts.key] = ts
for plugin in list(self.plugins.values()):
try:
plugin.transition(key, start, finish2, *args, **kwargs)
except Exception:
logger.info("Plugin failed with exception", exc_info=True)
if ts.state == "forgotten":
del self.tasks[ts.key]
tg: TaskGroup = ts.group
if ts.state == "forgotten" and tg.name in self.task_groups:
# Remove TaskGroup if all tasks are in the forgotten state
all_forgotten: bool = True
for s in ALL_TASK_STATES:
if tg.states.get(s):
all_forgotten = False
break
if all_forgotten:
ts.prefix.groups.remove(tg)
del self.task_groups[tg.name]
return recommendations, client_msgs, worker_msgs
except Exception:
logger.exception("Error transitioning %r from %r to %r", key, start, finish)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def _transitions(
self,
recommendations: dict,
client_msgs: dict,
worker_msgs: dict,
stimulus_id: str,
):
"""Process transitions until none are left
This includes feedback from previous transitions and continues until we
reach a steady state
"""
keys: set = set()
recommendations = recommendations.copy()
while recommendations:
key, finish = recommendations.popitem()
keys.add(key)
new = self._transition(key, finish, stimulus_id)
new_recs, new_cmsgs, new_wmsgs = new
recommendations.update(new_recs)
for c, new_msgs in new_cmsgs.items():
msgs = client_msgs.get(c)
if msgs is not None:
msgs.extend(new_msgs)
else:
client_msgs[c] = new_msgs
for w, new_msgs in new_wmsgs.items():
msgs = worker_msgs.get(w)
if msgs is not None:
msgs.extend(new_msgs)
else:
worker_msgs[w] = new_msgs
if self.validate:
# FIXME downcast antipattern
scheduler = cast(Scheduler, self)
for key in keys:
scheduler.validate_key(key)
def transition_released_waiting(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert ts.run_spec
assert not ts.waiting_on
assert not ts.who_has
assert not ts.processing_on
assert not any([dts.state == "forgotten" for dts in ts.dependencies])
if ts.has_lost_dependencies:
recommendations[key] = "forgotten"
return recommendations, client_msgs, worker_msgs
ts.state = "waiting"
dts: TaskState
for dts in ts.dependencies:
if dts.exception_blame:
ts.exception_blame = dts.exception_blame
recommendations[key] = "erred"
return recommendations, client_msgs, worker_msgs
for dts in ts.dependencies:
dep = dts.key
if not dts.who_has:
ts.waiting_on.add(dts)
if dts.state == "released":
recommendations[dep] = "waiting"
else:
dts.waiters.add(ts)
ts.waiters = {dts for dts in ts.dependents if dts.state == "waiting"}
if not ts.waiting_on:
if self.workers:
recommendations[key] = "processing"
else:
self.unrunnable.add(ts)
ts.state = "no-worker"
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_no_worker_waiting(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert ts in self.unrunnable
assert not ts.waiting_on
assert not ts.who_has
assert not ts.processing_on
self.unrunnable.remove(ts)
if ts.has_lost_dependencies:
recommendations[key] = "forgotten"
return recommendations, client_msgs, worker_msgs
for dts in ts.dependencies:
dep = dts.key
if not dts.who_has:
ts.waiting_on.add(dts)
if dts.state == "released":
recommendations[dep] = "waiting"
else:
dts.waiters.add(ts)
ts.state = "waiting"
if not ts.waiting_on:
if self.workers:
recommendations[key] = "processing"
else:
self.unrunnable.add(ts)
ts.state = "no-worker"
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_no_worker_memory(
self,
key,
stimulus_id,
nbytes=None,
type=None,
typename: str | None = None,
worker=None,
):
try:
ws: WorkerState = self.workers[worker]
ts: TaskState = self.tasks[key]
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert not ts.processing_on
assert not ts.waiting_on
assert ts.state == "no-worker"
self.unrunnable.remove(ts)
if nbytes is not None:
ts.set_nbytes(nbytes)
self.check_idle_saturated(ws)
_add_to_memory(
self, ts, ws, recommendations, client_msgs, type=type, typename=typename
)
ts.state = "memory"
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def decide_worker(self, ts: TaskState) -> WorkerState | None:
"""
Decide on a worker for task *ts*. Return a WorkerState.
If it's a root or root-like task, we place it with its relatives to
reduce future data tansfer.
If it has dependencies or restrictions, we use
`decide_worker_from_deps_and_restrictions`.
Otherwise, we pick the least occupied worker, or pick from all workers
in a round-robin fashion.
"""
if not self.workers:
return None
tg = ts.group
valid_workers = self.valid_workers(ts)
if (
valid_workers is not None
and not valid_workers
and not ts.loose_restrictions
):
self.unrunnable.add(ts)
ts.state = "no-worker"
return None
# Group is larger than cluster with few dependencies?
# Minimize future data transfers.
if (
valid_workers is None
and len(tg) > self.total_nthreads * 2
and len(tg.dependencies) < 5
and sum(map(len, tg.dependencies)) < 5
):
ws = tg.last_worker
if not (ws and tg.last_worker_tasks_left and ws.address in self.workers):
# Last-used worker is full or unknown; pick a new worker for the next few tasks
ws = min(
(self.idle or self.workers).values(),
key=partial(self.worker_objective, ts),
)
assert ws
tg.last_worker_tasks_left = math.floor(
(len(tg) / self.total_nthreads) * ws.nthreads
)
# Record `last_worker`, or clear it on the final task
tg.last_worker = (
ws if tg.states["released"] + tg.states["waiting"] > 1 else None
)
tg.last_worker_tasks_left -= 1
return ws
if ts.dependencies or valid_workers is not None:
ws = decide_worker(
ts,
self.workers.values(),
valid_workers,
partial(self.worker_objective, ts),
)
else:
# Fastpath when there are no related tasks or restrictions
worker_pool = self.idle or self.workers
wp_vals = worker_pool.values()
n_workers: int = len(wp_vals)
if n_workers < 20: # smart but linear in small case
ws = min(wp_vals, key=operator.attrgetter("occupancy"))
assert ws
if ws.occupancy == 0:
# special case to use round-robin; linear search
# for next worker with zero occupancy (or just
# land back where we started).
wp_i: WorkerState
start: int = self.n_tasks % n_workers
i: int
for i in range(n_workers):
wp_i = wp_vals[(i + start) % n_workers]
if wp_i.occupancy == 0:
ws = wp_i
break
else: # dumb but fast in large case
ws = wp_vals[self.n_tasks % n_workers]
if self.validate and ws is not None:
assert ws.address in self.workers
return ws
def transition_waiting_processing(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert not ts.waiting_on
assert not ts.who_has
assert not ts.exception_blame
assert not ts.processing_on
assert not ts.has_lost_dependencies
assert ts not in self.unrunnable
assert all(dts.who_has for dts in ts.dependencies)
ws = self.decide_worker(ts)
if ws is None:
return recommendations, client_msgs, worker_msgs
worker = ws.address
self._set_duration_estimate(ts, ws)
ts.processing_on = ws
ts.state = "processing"
self.consume_resources(ts, ws)
self.check_idle_saturated(ws)
self.n_tasks += 1
if ts.actor:
ws.actors.add(ts)
# logger.debug("Send job to worker: %s, %s", worker, key)
worker_msgs[worker] = [_task_to_msg(self, ts)]
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_waiting_memory(
self,
key,
stimulus_id,
nbytes=None,
type=None,
typename: str | None = None,
worker=None,
**kwargs,
):
try:
ws: WorkerState = self.workers[worker]
ts: TaskState = self.tasks[key]
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert not ts.processing_on
assert ts.waiting_on
assert ts.state == "waiting"
ts.waiting_on.clear()
if nbytes is not None:
ts.set_nbytes(nbytes)
self.check_idle_saturated(ws)
_add_to_memory(
self, ts, ws, recommendations, client_msgs, type=type, typename=typename
)
if self.validate:
assert not ts.processing_on
assert not ts.waiting_on
assert ts.who_has
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_processing_memory(
self,
key,
stimulus_id,
nbytes=None,
type=None,
typename: str | None = None,
worker=None,
startstops=None,
**kwargs,
):
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
try:
ts = self.tasks[key]
assert worker
assert isinstance(worker, str)
if self.validate:
assert ts.processing_on
ws = ts.processing_on
assert ts in ws.processing
assert not ts.waiting_on
assert not ts.who_has, (ts, ts.who_has)
assert not ts.exception_blame
assert ts.state == "processing"
ws = self.workers.get(worker)
if ws is None:
recommendations[key] = "released"
return recommendations, client_msgs, worker_msgs
if ws != ts.processing_on: # someone else has this task
logger.info(
"Unexpected worker completed task. Expected: %s, Got: %s, Key: %s",
ts.processing_on,
ws,
key,
)
assert ts.processing_on
worker_msgs[ts.processing_on.address] = [
{
"op": "cancel-compute",
"key": key,
"stimulus_id": stimulus_id,
}
]
#############################
# Update Timing Information #
#############################
if startstops:
startstop: dict
for startstop in startstops:
ts.group.add_duration(
stop=startstop["stop"],
start=startstop["start"],
action=startstop["action"],
)
s: set = self.unknown_durations.pop(ts.prefix.name, set())
tts: TaskState
steal = self.extensions.get("stealing")
for tts in s:
if tts.processing_on:
self._set_duration_estimate(tts, tts.processing_on)
if steal:
steal.recalculate_cost(tts)
############################
# Update State Information #
############################
if nbytes is not None:
ts.set_nbytes(nbytes)
_remove_from_processing(self, ts)
_add_to_memory(
self, ts, ws, recommendations, client_msgs, type=type, typename=typename
)
if self.validate:
assert not ts.processing_on
assert not ts.waiting_on
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_memory_released(self, key, stimulus_id, safe: bool = False):
ws: WorkerState
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert not ts.waiting_on
assert not ts.processing_on
if safe:
assert not ts.waiters
if ts.actor:
for ws in ts.who_has:
ws.actors.discard(ts)
if ts.who_wants:
ts.exception_blame = ts
ts.exception = "Worker holding Actor was lost"
recommendations[ts.key] = "erred"
return (
recommendations,
client_msgs,
worker_msgs,
) # don't try to recreate
# XXX factor this out?
worker_msg = {
"op": "free-keys",
"keys": [key],
"stimulus_id": stimulus_id,
}
for ws in ts.who_has:
worker_msgs[ws.address] = [worker_msg]
self.remove_all_replicas(ts)
ts.state = "released"
report_msg = {"op": "lost-data", "key": key}
cs: ClientState
for cs in ts.who_wants:
client_msgs[cs.client_key] = [report_msg]
if not ts.run_spec: # pure data
recommendations[key] = "forgotten"
elif ts.has_lost_dependencies:
recommendations[key] = "forgotten"
elif ts.who_wants or ts.waiters:
recommendations[key] = "waiting"
for dts in ts.waiters:
if dts.state in ("no-worker", "processing"):
recommendations[dts.key] = "waiting"
elif dts.state == "waiting":
dts.waiting_on.add(ts)
if self.validate:
assert not ts.waiting_on
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_released_erred(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
failing_ts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
with log_errors(pdb=LOG_PDB):
assert ts.exception_blame
assert not ts.who_has
assert not ts.waiting_on
assert not ts.waiters
failing_ts = ts.exception_blame
for dts in ts.dependents:
dts.exception_blame = failing_ts
if not dts.who_has:
recommendations[dts.key] = "erred"
report_msg = {
"op": "task-erred",
"key": key,
"exception": failing_ts.exception,
"traceback": failing_ts.traceback,
}
cs: ClientState
for cs in ts.who_wants:
client_msgs[cs.client_key] = [report_msg]
ts.state = "erred"
# TODO: waiting data?
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_erred_released(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
with log_errors(pdb=LOG_PDB):
assert ts.exception_blame
assert not ts.who_has
assert not ts.waiting_on
assert not ts.waiters
ts.exception = None
ts.exception_blame = None
ts.traceback = None
for dts in ts.dependents:
if dts.state == "erred":
recommendations[dts.key] = "waiting"
w_msg = {
"op": "free-keys",
"keys": [key],
"stimulus_id": stimulus_id,
}
for ws_addr in ts.erred_on:
worker_msgs[ws_addr] = [w_msg]
ts.erred_on.clear()
report_msg = {"op": "task-retried", "key": key}
cs: ClientState
for cs in ts.who_wants:
client_msgs[cs.client_key] = [report_msg]
ts.state = "released"
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_waiting_released(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert not ts.who_has
assert not ts.processing_on
dts: TaskState
for dts in ts.dependencies:
if ts in dts.waiters:
dts.waiters.discard(ts)
if not dts.waiters and not dts.who_wants:
recommendations[dts.key] = "released"
ts.waiting_on.clear()
ts.state = "released"
if ts.has_lost_dependencies:
recommendations[key] = "forgotten"
elif not ts.exception_blame and (ts.who_wants or ts.waiters):
recommendations[key] = "waiting"
else:
ts.waiters.clear()
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_processing_released(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert ts.processing_on
assert not ts.who_has
assert not ts.waiting_on
assert self.tasks[key].state == "processing"
w: str = _remove_from_processing(self, ts)
if w:
worker_msgs[w] = [
{
"op": "free-keys",
"keys": [key],
"stimulus_id": stimulus_id,
}
]
ts.state = "released"
if ts.has_lost_dependencies:
recommendations[key] = "forgotten"
elif ts.waiters or ts.who_wants:
recommendations[key] = "waiting"
if recommendations.get(key) != "waiting":
for dts in ts.dependencies:
if dts.state != "released":
dts.waiters.discard(ts)
if not dts.waiters and not dts.who_wants:
recommendations[dts.key] = "released"
ts.waiters.clear()
if self.validate:
assert not ts.processing_on
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_processing_erred(
self,
key: str,
stimulus_id: str,
cause: str | None = None,
exception=None,
traceback=None,
exception_text: str | None = None,
traceback_text: str | None = None,
worker: str | None = None,
**kwargs,
):
ws: WorkerState
try:
ts: TaskState = self.tasks[key]
dts: TaskState
failing_ts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert cause or ts.exception_blame
assert ts.processing_on
assert not ts.who_has
assert not ts.waiting_on
if ts.actor:
assert ts.processing_on
ws = ts.processing_on
ws.actors.remove(ts)
w = _remove_from_processing(self, ts)
ts.erred_on.add(w or worker) # type: ignore
if exception is not None:
ts.exception = exception
ts.exception_text = exception_text # type: ignore
if traceback is not None:
ts.traceback = traceback
ts.traceback_text = traceback_text # type: ignore
if cause is not None:
failing_ts = self.tasks[cause]
ts.exception_blame = failing_ts
else:
failing_ts = ts.exception_blame # type: ignore
for dts in ts.dependents:
dts.exception_blame = failing_ts
recommendations[dts.key] = "erred"
for dts in ts.dependencies:
dts.waiters.discard(ts)
if not dts.waiters and not dts.who_wants:
recommendations[dts.key] = "released"
ts.waiters.clear() # do anything with this?
ts.state = "erred"
report_msg = {
"op": "task-erred",
"key": key,
"exception": failing_ts.exception,
"traceback": failing_ts.traceback,
}
cs: ClientState
for cs in ts.who_wants:
client_msgs[cs.client_key] = [report_msg]
cs = self.clients["fire-and-forget"]
if ts in cs.wants_what:
_client_releases_keys(
self,
cs=cs,
keys=[key],
recommendations=recommendations,
)
if self.validate:
assert not ts.processing_on
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_no_worker_released(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
dts: TaskState
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert self.tasks[key].state == "no-worker"
assert not ts.who_has
assert not ts.waiting_on
self.unrunnable.remove(ts)
ts.state = "released"
for dts in ts.dependencies:
dts.waiters.discard(ts)
ts.waiters.clear()
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def _remove_key(self, key):
ts: TaskState = self.tasks.pop(key)
assert ts.state == "forgotten"
self.unrunnable.discard(ts)
cs: ClientState
for cs in ts.who_wants:
cs.wants_what.remove(ts)
ts.who_wants.clear()
ts.processing_on = None
ts.exception_blame = ts.exception = ts.traceback = None
self.task_metadata.pop(key, None)
def transition_memory_forgotten(self, key, stimulus_id):
ws: WorkerState
try:
ts: TaskState = self.tasks[key]
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert ts.state == "memory"
assert not ts.processing_on
assert not ts.waiting_on
if not ts.run_spec:
# It's ok to forget a pure data task
pass
elif ts.has_lost_dependencies:
# It's ok to forget a task with forgotten dependencies
pass
elif not ts.who_wants and not ts.waiters and not ts.dependents:
# It's ok to forget a task that nobody needs
pass
else:
assert 0, (ts,)
if ts.actor:
for ws in ts.who_has:
ws.actors.discard(ts)
_propagate_forgotten(self, ts, recommendations, worker_msgs, stimulus_id)
client_msgs = _task_to_client_msgs(ts)
self._remove_key(key)
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def transition_released_forgotten(self, key, stimulus_id):
try:
ts: TaskState = self.tasks[key]
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
if self.validate:
assert ts.state in ("released", "erred")
assert not ts.who_has
assert not ts.processing_on
assert not ts.waiting_on, (ts, ts.waiting_on)
if not ts.run_spec:
# It's ok to forget a pure data task
pass
elif ts.has_lost_dependencies:
# It's ok to forget a task with forgotten dependencies
pass
elif not ts.who_wants and not ts.waiters and not ts.dependents:
# It's ok to forget a task that nobody needs
pass
else:
assert 0, (ts,)
_propagate_forgotten(self, ts, recommendations, worker_msgs, stimulus_id)
client_msgs = _task_to_client_msgs(ts)
self._remove_key(key)
return recommendations, client_msgs, worker_msgs
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
# {
# (start, finish):
# transition_<start>_<finish>(
# self, key: str, stimulus_id: str, *args, **kwargs
# ) -> (recommendations, client_msgs, worker_msgs)
# }
_TRANSITIONS_TABLE: ClassVar[
Mapping[tuple[str, str], Callable[..., tuple[dict, dict, dict]]]
] = {
("released", "waiting"): transition_released_waiting,
("waiting", "released"): transition_waiting_released,
("waiting", "processing"): transition_waiting_processing,
("waiting", "memory"): transition_waiting_memory,
("processing", "released"): transition_processing_released,
("processing", "memory"): transition_processing_memory,
("processing", "erred"): transition_processing_erred,
("no-worker", "released"): transition_no_worker_released,
("no-worker", "waiting"): transition_no_worker_waiting,
("no-worker", "memory"): transition_no_worker_memory,
("released", "forgotten"): transition_released_forgotten,
("memory", "forgotten"): transition_memory_forgotten,
("erred", "released"): transition_erred_released,
("memory", "released"): transition_memory_released,
("released", "erred"): transition_released_erred,
}
##############################
# Assigning Tasks to Workers #
##############################
def _set_duration_estimate(self, ts: TaskState, ws: WorkerState) -> None:
"""Estimate task duration using worker state and task state.
If a task takes longer than twice the current average duration we
estimate the task duration to be 2x current-runtime, otherwise we set it
to be the average duration.
See also ``_remove_from_processing``
"""
# Long running tasks do not contribute to occupancy calculations and we
# do not set any task duration estimates
if ts in ws.long_running:
return
exec_time: float = ws.executing.get(ts, 0)
duration: float = self.get_task_duration(ts)
total_duration: float
if exec_time > 2 * duration:
total_duration = 2 * exec_time
else:
comm: float = self.get_comm_cost(ts, ws)
total_duration = duration + comm
old = ws.processing.get(ts, 0)
ws.processing[ts] = total_duration
self.total_occupancy += total_duration - old
ws.occupancy += total_duration - old
def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0):
"""Update the status of the idle and saturated state
The scheduler keeps track of workers that are ..
- Saturated: have enough work to stay busy
- Idle: do not have enough work to stay busy
They are considered saturated if they both have enough tasks to occupy
all of their threads, and if the expected runtime of those tasks is
large enough.
This is useful for load balancing and adaptivity.
"""
if self.total_nthreads == 0 or ws.status == Status.closed:
return
if occ < 0:
occ = ws.occupancy
nc: int = ws.nthreads
p: int = len(ws.processing)
avg: float = self.total_occupancy / self.total_nthreads
idle = self.idle
saturated = self.saturated
if p < nc or occ < nc * avg / 2:
idle[ws.address] = ws
saturated.discard(ws)
else:
idle.pop(ws.address, None)
if p > nc:
pending: float = occ * (p - nc) / (p * nc)
if 0.4 < pending > 1.9 * avg:
saturated.add(ws)
return
saturated.discard(ws)
def get_comm_cost(self, ts: TaskState, ws: WorkerState) -> float:
"""
Get the estimated communication cost (in s.) to compute the task
on the given worker.
"""
dts: TaskState
deps: set = ts.dependencies.difference(ws.has_what)
nbytes: int = 0
for dts in deps:
nbytes += dts.nbytes
return nbytes / self.bandwidth
def get_task_duration(self, ts: TaskState) -> float:
"""Get the estimated computation cost of the given task (not including
any communication cost).
If no data has been observed, value of
`distributed.scheduler.default-task-durations` are used. If none is set
for this task, `distributed.scheduler.unknown-task-duration` is used
instead.
"""
duration: float = ts.prefix.duration_average
if duration >= 0:
return duration
s = self.unknown_durations.get(ts.prefix.name)
if s is None:
self.unknown_durations[ts.prefix.name] = s = set()
s.add(ts)
return self.UNKNOWN_TASK_DURATION
def valid_workers(self, ts: TaskState) -> set[WorkerState] | None:
"""Return set of currently valid workers for key
If all workers are valid then this returns ``None``.
This checks tracks the following state:
* worker_restrictions
* host_restrictions
* resource_restrictions
"""
s: set | None = None
if ts.worker_restrictions:
s = {addr for addr in ts.worker_restrictions if addr in self.workers}
if ts.host_restrictions:
# Resolve the alias here rather than early, for the worker
# may not be connected when host_restrictions is populated
hr: list = [self.coerce_hostname(h) for h in ts.host_restrictions]
# XXX need HostState?
sl: list = []
for h in hr:
dh = self.host_info.get(h)
if dh is not None:
sl.append(dh["addresses"])
ss: set = set.union(*sl) if sl else set()
if s is None:
s = ss
else:
s |= ss
if ts.resource_restrictions:
dw = {}
for resource, required in ts.resource_restrictions.items():
dr = self.resources.get(resource)
if dr is None:
self.resources[resource] = dr = {}
sw: set = set()
for addr, supplied in dr.items():
if supplied >= required:
sw.add(addr)
dw[resource] = sw
ww: set = set.intersection(*dw.values())
if s is None:
s = ww
else:
s &= ww
if s is None:
if len(self.running) < len(self.workers):
return self.running.copy()
else:
s = {self.workers[addr] for addr in s}
if len(self.running) < len(self.workers):
s &= self.running
return s
def consume_resources(self, ts: TaskState, ws: WorkerState):
for r, required in ts.resource_restrictions.items():
ws.used_resources[r] += required
def release_resources(self, ts: TaskState, ws: WorkerState):
for r, required in ts.resource_restrictions.items():
ws.used_resources[r] -= required
def coerce_hostname(self, host):
"""
Coerce the hostname of a worker.
"""
alias = self.aliases.get(host)
if alias is not None:
ws: WorkerState = self.workers[alias]
return ws.host
else:
return host
def worker_objective(self, ts: TaskState, ws: WorkerState) -> tuple:
"""
Objective function to determine which worker should get the task
Minimize expected start time. If a tie then break with data storage.
"""
dts: TaskState
comm_bytes: int = 0
for dts in ts.dependencies:
if ws not in dts.who_has:
nbytes = dts.get_nbytes()
comm_bytes += nbytes
stack_time: float = ws.occupancy / ws.nthreads
start_time: float = stack_time + comm_bytes / self.bandwidth
if ts.actor:
return (len(ws.actors), start_time, ws.nbytes)
else:
return (start_time, ws.nbytes)
def add_replica(self, ts: TaskState, ws: WorkerState):
"""Note that a worker holds a replica of a task with state='memory'"""
if self.validate:
assert ws not in ts.who_has
assert ts not in ws.has_what
ws.nbytes += ts.get_nbytes()
ws._has_what[ts] = None
ts.who_has.add(ws)
if len(ts.who_has) == 2:
self.replicated_tasks.add(ts)
def remove_replica(self, ts: TaskState, ws: WorkerState):
"""Note that a worker no longer holds a replica of a task"""
ws.nbytes -= ts.get_nbytes()
del ws._has_what[ts]
ts.who_has.remove(ws)
if len(ts.who_has) == 1:
self.replicated_tasks.remove(ts)
def remove_all_replicas(self, ts: TaskState):
"""Remove all replicas of a task from all workers"""
ws: WorkerState
nbytes: int = ts.get_nbytes()
for ws in ts.who_has:
ws.nbytes -= nbytes
del ws._has_what[ts]
if len(ts.who_has) > 1:
self.replicated_tasks.remove(ts)
ts.who_has.clear()
def _reevaluate_occupancy_worker(self, ws: WorkerState):
"""See reevaluate_occupancy"""
ts: TaskState
old = ws.occupancy
for ts in ws.processing:
self._set_duration_estimate(ts, ws)
self.check_idle_saturated(ws)
steal = self.extensions.get("stealing")
if steal is None:
return
if ws.occupancy > old * 1.3 or old > ws.occupancy * 1.3:
for ts in ws.processing:
steal.recalculate_cost(ts)
def bulk_schedule_after_adding_worker(self, ws: WorkerState):
"""Send tasks with ts.state=='no-worker' in bulk to a worker that just joined.
Return recommendations. As the worker will start executing the new tasks
immediately, without waiting for the batch to end, we can't rely on worker-side
ordering, so the recommendations are sorted by priority order here.
"""
tasks = []
for ts in self.unrunnable:
valid = self.valid_workers(ts)
if valid is None or ws in valid:
tasks.append(ts)
# These recommendations will generate {"op": "compute-task"} messages
# to the worker in reversed order
tasks.sort(key=operator.attrgetter("priority"), reverse=True)
return {ts.key: "waiting" for ts in tasks}
[docs]class Scheduler(SchedulerState, ServerNode):
"""Dynamic distributed task scheduler
The scheduler tracks the current state of workers, data, and computations.
The scheduler listens for events and responds by controlling workers
appropriately. It continuously tries to use the workers to execute an ever
growing dask graph.
All events are handled quickly, in linear time with respect to their input
(which is often of constant size) and generally within a millisecond. To
accomplish this the scheduler tracks a lot of state. Every operation
maintains the consistency of this state.
The scheduler communicates with the outside world through Comm objects.
It maintains a consistent and valid view of the world even when listening
to several clients at once.
A Scheduler is typically started either with the ``dask-scheduler``
executable::
$ dask-scheduler
Scheduler started at 127.0.0.1:8786
Or within a LocalCluster a Client starts up without connection
information::
>>> c = Client() # doctest: +SKIP
>>> c.cluster.scheduler # doctest: +SKIP
Scheduler(...)
Users typically do not interact with the scheduler directly but rather with
the client object ``Client``.
The ``contact_address`` parameter allows to advertise a specific address to
the workers for communication with the scheduler, which is different than
the address the scheduler binds to. This is useful when the scheduler
listens on a private address, which therefore cannot be used by the workers
to contact it.
**State**
The scheduler contains the following state variables. Each variable is
listed along with what it stores and a brief description.
* **tasks:** ``{task key: TaskState}``
Tasks currently known to the scheduler
* **unrunnable:** ``{TaskState}``
Tasks in the "no-worker" state
* **workers:** ``{worker key: WorkerState}``
Workers currently connected to the scheduler
* **idle:** ``{WorkerState}``:
Set of workers that are not fully utilized
* **saturated:** ``{WorkerState}``:
Set of workers that are not over-utilized
* **host_info:** ``{hostname: dict}``:
Information about each worker host
* **clients:** ``{client key: ClientState}``
Clients currently connected to the scheduler
* **services:** ``{str: port}``:
Other services running on this scheduler, like Bokeh
* **loop:** ``IOLoop``:
The running Tornado IOLoop
* **client_comms:** ``{client key: Comm}``
For each client, a Comm object used to receive task requests and
report task status updates.
* **stream_comms:** ``{worker key: Comm}``
For each worker, a Comm object from which we both accept stimuli and
report results
* **task_duration:** ``{key-prefix: time}``
Time we expect certain functions to take, e.g. ``{'sum': 0.25}``
"""
default_port = 8786
_instances: ClassVar[weakref.WeakSet[Scheduler]] = weakref.WeakSet()
def __init__(
self,
loop=None,
delete_interval="500ms",
synchronize_worker_interval="60s",
services=None,
service_kwargs=None,
allowed_failures=None,
extensions=None,
validate=None,
scheduler_file=None,
security=None,
worker_ttl=None,
idle_timeout=None,
interface=None,
host=None,
port=0,
protocol=None,
dashboard_address=None,
dashboard=None,
http_prefix="/",
preload=None,
preload_argv=(),
plugins=(),
contact_address=None,
transition_counter_max=False,
**kwargs,
):
if loop is not None:
warnings.warn(
"the loop kwarg to Scheduler is deprecated",
DeprecationWarning,
stacklevel=2,
)
self.loop = self.io_loop = IOLoop.current()
self._setup_logging(logger)
# Attributes
if contact_address is None:
contact_address = dask.config.get("distributed.scheduler.contact-address")
self.contact_address = contact_address
if allowed_failures is None:
allowed_failures = dask.config.get("distributed.scheduler.allowed-failures")
self.allowed_failures = allowed_failures
if validate is None:
validate = dask.config.get("distributed.scheduler.validate")
self.proc = psutil.Process()
self.delete_interval = parse_timedelta(delete_interval, default="ms")
self.synchronize_worker_interval = parse_timedelta(
synchronize_worker_interval, default="ms"
)
self.digests = None
self.service_specs = services or {}
self.service_kwargs = service_kwargs or {}
self.services = {}
self.scheduler_file = scheduler_file
worker_ttl = worker_ttl or dask.config.get("distributed.scheduler.worker-ttl")
self.worker_ttl = parse_timedelta(worker_ttl) if worker_ttl else None
idle_timeout = idle_timeout or dask.config.get(
"distributed.scheduler.idle-timeout"
)
if idle_timeout:
self.idle_timeout = parse_timedelta(idle_timeout)
else:
self.idle_timeout = None
self.idle_since = time()
self.time_started = self.idle_since # compatibility for dask-gateway
self._lock = asyncio.Lock()
self.bandwidth_workers = defaultdict(float)
self.bandwidth_types = defaultdict(float)
if not preload:
preload = dask.config.get("distributed.scheduler.preload")
if not preload_argv:
preload_argv = dask.config.get("distributed.scheduler.preload-argv")
self.preloads = preloading.process_preloads(self, preload, preload_argv)
if isinstance(security, dict):
security = Security(**security)
self.security = security or Security()
assert isinstance(self.security, Security)
self.connection_args = self.security.get_connection_args("scheduler")
self.connection_args["handshake_overrides"] = { # common denominator
"pickle-protocol": 4
}
self._start_address = addresses_from_user_args(
host=host,
port=port,
interface=interface,
protocol=protocol,
security=security,
default_port=self.default_port,
)
http_server_modules = dask.config.get("distributed.scheduler.http.routes")
show_dashboard = dashboard or (dashboard is None and dashboard_address)
# install vanilla route if show_dashboard but bokeh is not installed
if show_dashboard:
try:
import distributed.dashboard.scheduler
except ImportError:
show_dashboard = False
http_server_modules.append("distributed.http.scheduler.missing_bokeh")
routes = get_handlers(
server=self, modules=http_server_modules, prefix=http_prefix
)
self.start_http_server(routes, dashboard_address, default_port=8787)
if show_dashboard:
distributed.dashboard.scheduler.connect(
self.http_application, self.http_server, self, prefix=http_prefix
)
# Communication state
self.client_comms = {}
self.stream_comms = {}
self._worker_coroutines = []
# Task state
tasks = {}
self.generation = 0
self._last_client = None
self._last_time = 0
unrunnable = set()
self.datasets = {}
# Prefix-keyed containers
# Client state
clients = {}
# Worker state
workers = SortedDict()
host_info = {}
resources = {}
aliases = {}
self._task_state_collections = [unrunnable]
self._worker_collections = [
workers,
host_info,
resources,
aliases,
]
self.transition_log = deque(
maxlen=dask.config.get("distributed.scheduler.transition-log-length")
)
self.log = deque(
maxlen=dask.config.get("distributed.scheduler.transition-log-length")
)
self.events = defaultdict(
partial(
deque, maxlen=dask.config.get("distributed.scheduler.events-log-length")
)
)
self.event_counts = defaultdict(int)
self.event_subscriber = defaultdict(set)
self.worker_plugins = {}
self.nanny_plugins = {}
worker_handlers = {
"task-finished": self.handle_task_finished,
"task-erred": self.handle_task_erred,
"release-worker-data": self.release_worker_data,
"add-keys": self.add_keys,
"long-running": self.handle_long_running,
"reschedule": self.reschedule,
"keep-alive": lambda *args, **kwargs: None,
"log-event": self.log_worker_event,
"worker-status-change": self.handle_worker_status_change,
"request-refresh-who-has": self.handle_request_refresh_who_has,
}
client_handlers = {
"update-graph": self.update_graph,
"update-graph-hlg": self.update_graph_hlg,
"client-desires-keys": self.client_desires_keys,
"update-data": self.update_data,
"report-key": self.report_on_key,
"client-releases-keys": self.client_releases_keys,
"heartbeat-client": self.client_heartbeat,
"close-client": self.remove_client,
"restart": self.restart,
"subscribe-topic": self.subscribe_topic,
"unsubscribe-topic": self.unsubscribe_topic,
}
self.handlers = {
"register-client": self.add_client,
"scatter": self.scatter,
"register-worker": self.add_worker,
"register_nanny": self.add_nanny,
"unregister": self.remove_worker,
"gather": self.gather,
"cancel": self.stimulus_cancel,
"retry": self.stimulus_retry,
"feed": self.feed,
"terminate": self.close,
"broadcast": self.broadcast,
"proxy": self.proxy,
"ncores": self.get_ncores,
"ncores_running": self.get_ncores_running,
"has_what": self.get_has_what,
"who_has": self.get_who_has,
"processing": self.get_processing,
"call_stack": self.get_call_stack,
"profile": self.get_profile,
"performance_report": self.performance_report,
"get_logs": self.get_logs,
"logs": self.get_logs,
"worker_logs": self.get_worker_logs,
"log_event": self.log_event,
"events": self.get_events,
"nbytes": self.get_nbytes,
"versions": self.versions,
"add_keys": self.add_keys,
"rebalance": self.rebalance,
"replicate": self.replicate,
"run_function": self.run_function,
"update_data": self.update_data,
"set_resources": self.add_resources,
"retire_workers": self.retire_workers,
"get_metadata": self.get_metadata,
"set_metadata": self.set_metadata,
"set_restrictions": self.set_restrictions,
"heartbeat_worker": self.heartbeat_worker,
"get_task_status": self.get_task_status,
"get_task_stream": self.get_task_stream,
"get_task_prefix_states": self.get_task_prefix_states,
"register_scheduler_plugin": self.register_scheduler_plugin,
"register_worker_plugin": self.register_worker_plugin,
"unregister_worker_plugin": self.unregister_worker_plugin,
"register_nanny_plugin": self.register_nanny_plugin,
"unregister_nanny_plugin": self.unregister_nanny_plugin,
"adaptive_target": self.adaptive_target,
"workers_to_close": self.workers_to_close,
"subscribe_worker_status": self.subscribe_worker_status,
"start_task_metadata": self.start_task_metadata,
"stop_task_metadata": self.stop_task_metadata,
"get_cluster_state": self.get_cluster_state,
"dump_cluster_state_to_url": self.dump_cluster_state_to_url,
"benchmark_hardware": self.benchmark_hardware,
"get_story": self.get_story,
}
connection_limit = get_fileno_limit() / 2
SchedulerState.__init__(
self,
aliases=aliases,
clients=clients,
workers=workers,
host_info=host_info,
resources=resources,
tasks=tasks,
unrunnable=unrunnable,
validate=validate,
plugins=plugins,
transition_counter_max=transition_counter_max,
)
ServerNode.__init__(
self,
handlers=self.handlers,
stream_handlers=merge(worker_handlers, client_handlers),
connection_limit=connection_limit,
deserialize=False,
connection_args=self.connection_args,
**kwargs,
)
if self.worker_ttl:
pc = PeriodicCallback(self.check_worker_ttl, self.worker_ttl * 1000)
self.periodic_callbacks["worker-ttl"] = pc
if self.idle_timeout:
pc = PeriodicCallback(self.check_idle, self.idle_timeout * 1000 / 4)
self.periodic_callbacks["idle-timeout"] = pc
if extensions is None:
extensions = DEFAULT_EXTENSIONS.copy()
if not dask.config.get("distributed.scheduler.work-stealing"):
if "stealing" in extensions:
del extensions["stealing"]
for name, extension in extensions.items():
self.extensions[name] = extension(self)
setproctitle("dask-scheduler [not started]")
Scheduler._instances.add(self)
self.rpc.allow_offload = False
##################
# Administration #
##################
def __repr__(self):
return (
f"<Scheduler {self.address_safe!r}, "
f"workers: {len(self.workers)}, "
f"cores: {self.total_nthreads}, "
f"tasks: {len(self.tasks)}>"
)
def _repr_html_(self):
return get_template("scheduler.html.j2").render(
address=self.address,
workers=self.workers,
threads=self.total_nthreads,
tasks=self.tasks,
)
[docs] def identity(self):
"""Basic information about ourselves and our cluster"""
d = {
"type": type(self).__name__,
"id": str(self.id),
"address": self.address,
"services": {key: v.port for (key, v) in self.services.items()},
"started": self.time_started,
"workers": {
worker.address: worker.identity() for worker in self.workers.values()
},
}
return d
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Server.identity
Client.dump_cluster_state
distributed.utils.recursive_to_dict
"""
info = super()._to_dict(exclude=exclude)
extra = {
"transition_log": self.transition_log,
"transition_counter": self.transition_counter,
"log": self.log,
"tasks": self.tasks,
"task_groups": self.task_groups,
# Overwrite dict of WorkerState.identity from info
"workers": self.workers,
"clients": self.clients,
"memory": self.memory,
"events": self.events,
"extensions": self.extensions,
}
extra = {k: v for k, v in extra.items() if k not in exclude}
info.update(recursive_to_dict(extra, exclude=exclude))
return info
[docs] async def get_cluster_state(
self,
exclude: "Collection[str]",
) -> dict:
"Produce the state dict used in a cluster state dump"
# Kick off state-dumping on workers before we block the event loop in `self._to_dict`.
workers_future = asyncio.gather(
self.broadcast(
msg={"op": "dump_state", "exclude": exclude},
on_error="return",
),
self.broadcast(
msg={"op": "versions"},
on_error="ignore",
),
)
try:
scheduler_state = self._to_dict(exclude=exclude)
worker_states, worker_versions = await workers_future
finally:
# Ensure the tasks aren't left running if anything fails.
# Someday (py3.11), use a trio-style TaskGroup for this.
workers_future.cancel()
# Convert any RPC errors to strings
worker_states = {
k: repr(v) if isinstance(v, Exception) else v
for k, v in worker_states.items()
}
return {
"scheduler": scheduler_state,
"workers": worker_states,
"versions": {"scheduler": self.versions(), "workers": worker_versions},
}
[docs] async def dump_cluster_state_to_url(
self,
url: str,
exclude: "Collection[str]",
format: Literal["msgpack", "yaml"],
**storage_options: dict[str, Any],
) -> None:
"Write a cluster state dump to an fsspec-compatible URL."
await cluster_dump.write_state(
partial(self.get_cluster_state, exclude), url, format, **storage_options
)
[docs] def get_worker_service_addr(
self, worker: str, service_name: str, protocol: bool = False
) -> tuple[str, int] | str | None:
"""
Get the (host, port) address of the named service on the *worker*.
Returns None if the service doesn't exist.
Parameters
----------
worker : address
service_name : str
Common services include 'bokeh' and 'nanny'
protocol : boolean
Whether or not to include a full address with protocol (True)
or just a (host, port) pair
"""
ws = self.workers[worker]
port = ws.services.get(service_name)
if port is None:
return None
elif protocol:
return "%(protocol)s://%(host)s:%(port)d" % {
"protocol": ws.address.split("://")[0],
"host": ws.host,
"port": port,
}
else:
return ws.host, port
[docs] async def start_unsafe(self):
"""Clear out old state and restart all running coroutines"""
await super().start_unsafe()
enable_gc_diagnosis()
self.clear_task_state()
with suppress(AttributeError):
for c in self._worker_coroutines:
c.cancel()
for addr in self._start_address:
await self.listen(
addr,
allow_offload=False,
handshake_overrides={"pickle-protocol": 4, "compression": None},
**self.security.get_listen_args("scheduler"),
)
self.ip = get_address_host(self.listen_address)
listen_ip = self.ip
if listen_ip == "0.0.0.0":
listen_ip = ""
if self.address.startswith("inproc://"):
listen_ip = "localhost"
# Services listen on all addresses
self.start_services(listen_ip)
for listener in self.listeners:
logger.info(" Scheduler at: %25s", listener.contact_address)
for k, v in self.services.items():
logger.info("%11s at: %25s", k, "%s:%d" % (listen_ip, v.port))
self._ongoing_background_tasks.call_soon(self.reevaluate_occupancy)
if self.scheduler_file:
with open(self.scheduler_file, "w") as f:
json.dump(self.identity(), f, indent=2)
fn = self.scheduler_file # remove file when we close the process
def del_scheduler_file():
if os.path.exists(fn):
os.remove(fn)
weakref.finalize(self, del_scheduler_file)
for preload in self.preloads:
try:
await preload.start()
except Exception:
logger.exception("Failed to start preload")
await asyncio.gather(
*[plugin.start(self) for plugin in list(self.plugins.values())]
)
self.start_periodic_callbacks()
setproctitle(f"dask-scheduler [{self.address}]")
return self
[docs] async def close(self, fast=None, close_workers=None):
"""Send cleanup signal to all coroutines then wait until finished
See Also
--------
Scheduler.cleanup
"""
if fast is not None or close_workers is not None:
warnings.warn(
"The 'fast' and 'close_workers' parameters in Scheduler.close have no effect and will be removed in a future version of distributed.",
FutureWarning,
)
if self.status in (Status.closing, Status.closed):
await self.finished()
return
async def log_errors(func):
try:
await func()
except Exception:
logger.exception("Plugin call failed during scheduler.close")
await asyncio.gather(
*[log_errors(plugin.before_close) for plugin in list(self.plugins.values())]
)
self.status = Status.closing
logger.info("Scheduler closing...")
setproctitle("dask-scheduler [closing]")
for preload in self.preloads:
try:
await preload.teardown()
except Exception:
logger.exception("Failed to tear down preload")
await asyncio.gather(
*[log_errors(plugin.close) for plugin in list(self.plugins.values())]
)
for pc in self.periodic_callbacks.values():
pc.stop()
self.periodic_callbacks.clear()
self.stop_services()
for ext in self.extensions.values():
with suppress(AttributeError):
ext.teardown()
logger.info("Scheduler closing all comms")
futures = []
for _, comm in list(self.stream_comms.items()):
# FIXME use `self.remove_worker()` instead after https://github.com/dask/distributed/issues/6390
if not comm.closed():
# This closes the Worker and ensures that if a Nanny is around,
# it is closed as well
comm.send({"op": "close"})
comm.send({"op": "close-stream"})
# ^ TODO remove? `Worker.close` will close the stream anyway.
with suppress(AttributeError):
futures.append(comm.close())
await asyncio.gather(*futures)
for comm in self.client_comms.values():
comm.abort()
await self.rpc.close()
self.status = Status.closed
self.stop()
await super().close()
setproctitle("dask-scheduler [closed]")
disable_gc_diagnosis()
[docs] @log_errors
async def close_worker(self, worker: str, stimulus_id: str, safe: bool = False):
"""Remove a worker from the cluster
This both removes the worker from our local state and also sends a
signal to the worker to shut down. This works regardless of whether or
not the worker has a nanny process restarting it
"""
logger.info("Closing worker %s", worker)
self.log_event(worker, {"action": "close-worker"})
self.worker_send(worker, {"op": "close"}) # TODO redundant with `remove_worker`
await self.remove_worker(address=worker, safe=safe, stimulus_id=stimulus_id)
###########
# Stimuli #
###########
def heartbeat_worker(
self,
comm=None,
*,
address,
resolve_address: bool = True,
now: float | None = None,
resources: dict[str, float] | None = None,
host_info: dict | None = None,
metrics: dict,
executing: dict[str, float] | None = None,
extensions: dict | None = None,
) -> dict[str, Any]:
address = self.coerce_address(address, resolve_address)
address = normalize_address(address)
ws = self.workers.get(address)
if ws is None:
logger.warning(f"Received heartbeat from unregistered worker {address!r}.")
return {"status": "missing"}
host = get_address_host(address)
local_now = time()
host_info = host_info or {}
dh: dict = self.host_info.setdefault(host, {})
dh["last-seen"] = local_now
frac = 1 / len(self.workers)
self.bandwidth = (
self.bandwidth * (1 - frac) + metrics["bandwidth"]["total"] * frac
)
for other, (bw, count) in metrics["bandwidth"]["workers"].items():
if (address, other) not in self.bandwidth_workers:
self.bandwidth_workers[address, other] = bw / count
else:
alpha = (1 - frac) ** count
self.bandwidth_workers[address, other] = self.bandwidth_workers[
address, other
] * alpha + bw * (1 - alpha)
for typ, (bw, count) in metrics["bandwidth"]["types"].items():
if typ not in self.bandwidth_types:
self.bandwidth_types[typ] = bw / count
else:
alpha = (1 - frac) ** count
self.bandwidth_types[typ] = self.bandwidth_types[typ] * alpha + bw * (
1 - alpha
)
ws.last_seen = local_now
if executing is not None:
ws.executing = {
self.tasks[key]: duration
for key, duration in executing.items()
if key in self.tasks
}
ws.metrics = metrics
# Calculate RSS - dask keys, separating "old" and "new" usage
# See MemoryState for details
max_memory_unmanaged_old_hist_age = local_now - self.MEMORY_RECENT_TO_OLD_TIME
memory_unmanaged_old = ws._memory_unmanaged_old
while ws._memory_unmanaged_history:
timestamp, size = ws._memory_unmanaged_history[0]
if timestamp >= max_memory_unmanaged_old_hist_age:
break
ws._memory_unmanaged_history.popleft()
if size == memory_unmanaged_old:
memory_unmanaged_old = 0 # recalculate min()
# metrics["memory"] is None if the worker sent a heartbeat before its
# SystemMonitor ever had a chance to run.
# ws._nbytes is updated at a different time and sizeof() may not be accurate,
# so size may be (temporarily) negative; floor it to zero.
size = max(
0,
(metrics["memory"] or 0) - ws.nbytes + metrics["spilled_nbytes"]["memory"],
)
ws._memory_unmanaged_history.append((local_now, size))
if not memory_unmanaged_old:
# The worker has just been started or the previous minimum has been expunged
# because too old.
# Note: this algorithm is capped to 200 * MEMORY_RECENT_TO_OLD_TIME elements
# cluster-wide by heartbeat_interval(), regardless of the number of workers
ws._memory_unmanaged_old = min(map(second, ws._memory_unmanaged_history))
elif size < memory_unmanaged_old:
ws._memory_unmanaged_old = size
if host_info:
dh = self.host_info.setdefault(host, {})
dh.update(host_info)
if now:
ws.time_delay = local_now - now
if resources:
self.add_resources(worker=address, resources=resources)
if extensions:
for name, data in extensions.items():
self.extensions[name].heartbeat(ws, data)
return {
"status": "OK",
"time": local_now,
"heartbeat-interval": heartbeat_interval(len(self.workers)),
}
[docs] @log_errors
async def add_worker(
self,
comm=None,
*,
address: str,
status: str,
server_id: str,
keys=(),
nthreads=None,
name=None,
resolve_address=True,
nbytes=None,
types=None,
now=None,
resources=None,
host_info=None,
memory_limit=None,
metrics=None,
pid=0,
services=None,
local_directory=None,
versions: dict[str, Any] | None = None,
nanny=None,
extra=None,
stimulus_id=None,
):
"""Add a new worker to the cluster"""
address = self.coerce_address(address, resolve_address)
address = normalize_address(address)
host = get_address_host(address)
if address in self.workers:
raise ValueError("Worker already exists %s" % address)
if nbytes:
err = (
f"Worker {address!r} connected with {len(nbytes)} key(s) in memory! Worker reconnection is not supported. "
f"Keys: {list(nbytes)}"
)
logger.error(err)
if comm:
await comm.write({"status": "error", "message": err, "time": time()})
return
if name in self.aliases:
logger.warning("Worker tried to connect with a duplicate name: %s", name)
msg = {
"status": "error",
"message": "name taken, %s" % name,
"time": time(),
}
if comm:
await comm.write(msg)
return
self.log_event(address, {"action": "add-worker"})
self.log_event("all", {"action": "add-worker", "worker": address})
self.workers[address] = ws = WorkerState(
address=address,
status=Status.lookup[status], # type: ignore
pid=pid,
nthreads=nthreads,
memory_limit=memory_limit or 0,
name=name,
local_directory=local_directory,
services=services,
versions=versions,
nanny=nanny,
extra=extra,
server_id=server_id,
)
if ws.status == Status.running:
self.running.add(ws)
dh = self.host_info.get(host)
if dh is None:
self.host_info[host] = dh = {}
dh_addresses = dh.get("addresses")
if dh_addresses is None:
dh["addresses"] = dh_addresses = set()
dh["nthreads"] = 0
dh_addresses.add(address)
dh["nthreads"] += nthreads
self.total_nthreads += nthreads
self.aliases[name] = address
self.heartbeat_worker(
address=address,
resolve_address=resolve_address,
now=now,
resources=resources,
host_info=host_info,
metrics=metrics,
)
# Do not need to adjust self.total_occupancy as self.occupancy[ws] cannot
# exist before this.
self.check_idle_saturated(ws)
# for key in keys: # TODO
# self.mark_key_in_memory(key, [address])
self.stream_comms[address] = BatchedSend(interval="5ms", loop=self.loop)
if ws.nthreads > len(ws.processing):
self.idle[ws.address] = ws
for plugin in list(self.plugins.values()):
try:
result = plugin.add_worker(scheduler=self, worker=address)
if result is not None and inspect.isawaitable(result):
await result
except Exception as e:
logger.exception(e)
if ws.status == Status.running:
self.transitions(self.bulk_schedule_after_adding_worker(ws), stimulus_id)
logger.info("Register worker %s", ws)
msg = {
"status": "OK",
"time": time(),
"heartbeat-interval": heartbeat_interval(len(self.workers)),
"worker-plugins": self.worker_plugins,
}
cs: ClientState
version_warning = version_module.error_message(
version_module.get_versions(),
merge(
{w: ws.versions for w, ws in self.workers.items()},
{c: cs.versions for c, cs in self.clients.items() if cs.versions},
),
versions,
client_name="This Worker",
)
msg.update(version_warning)
if comm:
await comm.write(msg)
await self.handle_worker(comm=comm, worker=address, stimulus_id=stimulus_id)
async def add_nanny(self, comm):
msg = {
"status": "OK",
"nanny-plugins": self.nanny_plugins,
}
return msg
def update_graph_hlg(
self,
client=None,
hlg=None,
keys=None,
dependencies=None,
restrictions=None,
priority=None,
loose_restrictions=None,
resources=None,
submitting_task=None,
retries=None,
user_priority=0,
actors=None,
fifo_timeout=0,
code=None,
):
unpacked_graph = HighLevelGraph.__dask_distributed_unpack__(hlg)
dsk = unpacked_graph["dsk"]
dependencies = unpacked_graph["deps"]
annotations = unpacked_graph["annotations"]
# Remove any self-dependencies (happens on test_publish_bag() and others)
for k, v in dependencies.items():
deps = set(v)
if k in deps:
deps.remove(k)
dependencies[k] = deps
if priority is None:
# Removing all non-local keys before calling order()
dsk_keys = set(dsk) # intersection() of sets is much faster than dict_keys
stripped_deps = {
k: v.intersection(dsk_keys)
for k, v in dependencies.items()
if k in dsk_keys
}
priority = dask.order.order(dsk, dependencies=stripped_deps)
return self.update_graph(
client,
dsk,
keys,
dependencies,
restrictions,
priority,
loose_restrictions,
resources,
submitting_task,
retries,
user_priority,
actors,
fifo_timeout,
annotations,
code=code,
stimulus_id=f"update-graph-{time()}",
)
[docs] def update_graph(
self,
client=None,
tasks=None,
keys=None,
dependencies=None,
restrictions=None,
priority=None,
loose_restrictions=None,
resources=None,
submitting_task=None,
retries=None,
user_priority=0,
actors=None,
fifo_timeout=0,
annotations=None,
code=None,
stimulus_id=None,
):
"""
Add new computations to the internal dask graph
This happens whenever the Client calls submit, map, get, or compute.
"""
stimulus_id = stimulus_id or f"update-graph-{time()}"
start = time()
fifo_timeout = parse_timedelta(fifo_timeout)
keys = set(keys)
if len(tasks) > 1:
self.log_event(
["all", client], {"action": "update_graph", "count": len(tasks)}
)
# Remove aliases
for k in list(tasks):
if tasks[k] is k:
del tasks[k]
dependencies = dependencies or {}
if self.total_occupancy > 1e-9 and self.computations:
# Still working on something. Assign new tasks to same computation
computation = self.computations[-1]
else:
computation = Computation()
self.computations.append(computation)
if code and code not in computation.code: # add new code blocks
computation.code.add(code)
n = 0
while len(tasks) != n: # walk through new tasks, cancel any bad deps
n = len(tasks)
for k, deps in list(dependencies.items()):
if any(
dep not in self.tasks and dep not in tasks for dep in deps
): # bad key
logger.info("User asked for computation on lost data, %s", k)
del tasks[k]
del dependencies[k]
if k in keys:
keys.remove(k)
self.report({"op": "cancelled-key", "key": k}, client=client)
self.client_releases_keys(
keys=[k], client=client, stimulus_id=stimulus_id
)
# Avoid computation that is already finished
already_in_memory = set() # tasks that are already done
for k, v in dependencies.items():
if v and k in self.tasks:
ts = self.tasks[k]
if ts.state in ("memory", "erred"):
already_in_memory.add(k)
if already_in_memory:
dependents = dask.core.reverse_dict(dependencies)
stack = list(already_in_memory)
done = set(already_in_memory)
while stack: # remove unnecessary dependencies
key = stack.pop()
try:
deps = dependencies[key]
except KeyError:
deps = self.tasks[key].dependencies
for dep in deps:
if dep in dependents:
child_deps = dependents[dep]
elif dep in self.tasks:
child_deps = self.tasks[dep].dependencies
else:
child_deps = set()
if all(d in done for d in child_deps):
if dep in self.tasks and dep not in done:
done.add(dep)
stack.append(dep)
for d in done:
tasks.pop(d, None)
dependencies.pop(d, None)
# Get or create task states
stack = list(keys)
touched_keys = set()
touched_tasks = []
while stack:
k = stack.pop()
if k in touched_keys:
continue
# XXX Have a method get_task_state(self, k) ?
ts = self.tasks.get(k)
if ts is None:
ts = self.new_task(k, tasks.get(k), "released", computation=computation)
elif not ts.run_spec:
ts.run_spec = tasks.get(k)
touched_keys.add(k)
touched_tasks.append(ts)
stack.extend(dependencies.get(k, ()))
self.client_desires_keys(keys=keys, client=client)
# Add dependencies
for key, deps in dependencies.items():
ts = self.tasks.get(key)
if ts is None or ts.dependencies:
continue
for dep in deps:
dts = self.tasks[dep]
ts.add_dependency(dts)
# Compute priorities
if isinstance(user_priority, Number):
user_priority = {k: user_priority for k in tasks}
annotations = annotations or {}
restrictions = restrictions or {}
loose_restrictions = loose_restrictions or []
resources = resources or {}
retries = retries or {}
# Override existing taxonomy with per task annotations
if annotations:
if "priority" in annotations:
user_priority.update(annotations["priority"])
if "workers" in annotations:
restrictions.update(annotations["workers"])
if "allow_other_workers" in annotations:
loose_restrictions.extend(
k for k, v in annotations["allow_other_workers"].items() if v
)
if "retries" in annotations:
retries.update(annotations["retries"])
if "resources" in annotations:
resources.update(annotations["resources"])
for a, kv in annotations.items():
for k, v in kv.items():
# Tasks might have been culled, in which case
# we have nothing to annotate.
ts = self.tasks.get(k)
if ts is not None:
ts.annotations[a] = v
# Add actors
if actors is True:
actors = list(keys)
for actor in actors or []:
ts = self.tasks[actor]
ts.actor = True
priority = priority or dask.order.order(
tasks
) # TODO: define order wrt old graph
if submitting_task: # sub-tasks get better priority than parent tasks
ts = self.tasks.get(submitting_task)
if ts is not None:
generation = ts.priority[0] - 0.01
else: # super-task already cleaned up
generation = self.generation
elif self._last_time + fifo_timeout < start:
self.generation += 1 # older graph generations take precedence
generation = self.generation
self._last_time = start
else:
generation = self.generation
for key in set(priority) & touched_keys:
ts = self.tasks[key]
if ts.priority is None:
ts.priority = (-(user_priority.get(key, 0)), generation, priority[key])
# Ensure all runnables have a priority
runnables = [ts for ts in touched_tasks if ts.run_spec]
for ts in runnables:
if ts.priority is None and ts.run_spec:
ts.priority = (self.generation, 0)
if restrictions:
# *restrictions* is a dict keying task ids to lists of
# restriction specifications (either worker names or addresses)
for k, v in restrictions.items():
if v is None:
continue
ts = self.tasks.get(k)
if ts is None:
continue
ts.host_restrictions = set()
ts.worker_restrictions = set()
# Make sure `v` is a collection and not a single worker name / address
if not isinstance(v, (list, tuple, set)):
v = [v]
for w in v:
try:
w = self.coerce_address(w)
except ValueError:
# Not a valid address, but perhaps it's a hostname
ts.host_restrictions.add(w)
else:
ts.worker_restrictions.add(w)
if loose_restrictions:
for k in loose_restrictions:
ts = self.tasks[k]
ts.loose_restrictions = True
if resources:
for k, v in resources.items():
if v is None:
continue
assert isinstance(v, dict)
ts = self.tasks.get(k)
if ts is None:
continue
ts.resource_restrictions = v
if retries:
for k, v in retries.items():
assert isinstance(v, int)
ts = self.tasks.get(k)
if ts is None:
continue
ts.retries = v
# Compute recommendations
recommendations: dict = {}
for ts in sorted(runnables, key=operator.attrgetter("priority"), reverse=True):
if ts.state == "released" and ts.run_spec:
recommendations[ts.key] = "waiting"
for ts in touched_tasks:
for dts in ts.dependencies:
if dts.exception_blame:
ts.exception_blame = dts.exception_blame
recommendations[ts.key] = "erred"
break
for plugin in list(self.plugins.values()):
try:
plugin.update_graph(
self,
client=client,
tasks=tasks,
keys=keys,
restrictions=restrictions or {},
dependencies=dependencies,
priority=priority,
loose_restrictions=loose_restrictions,
resources=resources,
annotations=annotations,
)
except Exception as e:
logger.exception(e)
self.transitions(recommendations, stimulus_id)
for ts in touched_tasks:
if ts.state in ("memory", "erred"):
self.report_on_key(ts=ts, client=client)
end = time()
if self.digests is not None:
self.digests["update-graph-duration"].add(end - start)
# TODO: balance workers
[docs] def stimulus_task_finished(self, key=None, worker=None, stimulus_id=None, **kwargs):
"""Mark that a task has finished execution on a particular worker"""
logger.debug("Stimulus task finished %s, %s", key, worker)
recommendations: dict = {}
client_msgs: dict = {}
worker_msgs: dict = {}
ws: WorkerState = self.workers[worker]
ts: TaskState = self.tasks.get(key)
if ts is None or ts.state == "released":
logger.debug(
"Received already computed task, worker: %s, state: %s"
", key: %s, who_has: %s",
worker,
ts.state if ts else "forgotten",
key,
ts.who_has if ts else {},
)
worker_msgs[worker] = [
{
"op": "free-keys",
"keys": [key],
"stimulus_id": stimulus_id,
}
]
elif ts.state == "memory":
self.add_keys(worker=worker, keys=[key])
else:
ts.metadata.update(kwargs["metadata"])
r: tuple = self._transition(
key, "memory", stimulus_id, worker=worker, **kwargs
)
recommendations, client_msgs, worker_msgs = r
if ts.state == "memory":
assert ws in ts.who_has
return recommendations, client_msgs, worker_msgs
[docs] def stimulus_task_erred(
self,
key=None,
worker=None,
exception=None,
stimulus_id=None,
traceback=None,
**kwargs,
):
"""Mark that a task has erred on a particular worker"""
logger.debug("Stimulus task erred %s, %s", key, worker)
ts: TaskState = self.tasks.get(key)
if ts is None or ts.state != "processing":
return {}, {}, {}
if ts.retries > 0:
ts.retries -= 1
return self._transition(key, "waiting", stimulus_id)
else:
return self._transition(
key,
"erred",
stimulus_id,
cause=key,
exception=exception,
traceback=traceback,
worker=worker,
**kwargs,
)
def stimulus_retry(self, keys, client=None):
logger.info("Client %s requests to retry %d keys", client, len(keys))
if client:
self.log_event(client, {"action": "retry", "count": len(keys)})
stack = list(keys)
seen = set()
roots = []
ts: TaskState
dts: TaskState
while stack:
key = stack.pop()
seen.add(key)
ts = self.tasks[key]
erred_deps = [dts.key for dts in ts.dependencies if dts.state == "erred"]
if erred_deps:
stack.extend(erred_deps)
else:
roots.append(key)
recommendations: dict = {key: "waiting" for key in roots}
self.transitions(recommendations, f"stimulus-retry-{time()}")
if self.validate:
for key in seen:
assert not self.tasks[key].exception_blame
return tuple(seen)
[docs] @log_errors
async def remove_worker(
self, address: str, *, stimulus_id: str, safe: bool = False, close: bool = True
) -> Literal["OK", "already-removed"]:
"""
Remove worker from cluster
We do this when a worker reports that it plans to leave or when it
appears to be unresponsive. This may send its tasks back to a released
state.
"""
if self.status == Status.closed:
return "already-removed"
address = self.coerce_address(address)
if address not in self.workers:
return "already-removed"
host = get_address_host(address)
ws: WorkerState = self.workers[address]
event_msg = {
"action": "remove-worker",
"processing-tasks": {ts.key: cost for ts, cost in ws.processing.items()},
}
self.log_event(address, event_msg.copy())
event_msg["worker"] = address
self.log_event("all", event_msg)
logger.info("Remove worker %s", ws)
if close:
with suppress(AttributeError, CommClosedError):
self.stream_comms[address].send({"op": "close"})
self.remove_resources(address)
dh: dict = self.host_info[host]
dh_addresses: set = dh["addresses"]
dh_addresses.remove(address)
dh["nthreads"] -= ws.nthreads
self.total_nthreads -= ws.nthreads
if not dh_addresses:
del self.host_info[host]
self.rpc.remove(address)
del self.stream_comms[address]
del self.aliases[ws.name]
self.idle.pop(ws.address, None)
self.saturated.discard(ws)
del self.workers[address]
ws.status = Status.closed
self.running.discard(ws)
self.total_occupancy -= ws.occupancy
recommendations: dict = {}
ts: TaskState
for ts in list(ws.processing):
k = ts.key
recommendations[k] = "released"
if not safe:
ts.suspicious += 1
ts.prefix.suspicious += 1
if ts.suspicious > self.allowed_failures:
del recommendations[k]
e = pickle.dumps(
KilledWorker(task=k, last_worker=ws.clean()), protocol=4
)
r = self.transition(
k, "erred", exception=e, cause=k, stimulus_id=stimulus_id
)
recommendations.update(r)
logger.info(
"Task %s marked as failed because %d workers died"
" while trying to run it",
ts.key,
self.allowed_failures,
)
for ts in list(ws.has_what):
self.remove_replica(ts, ws)
if not ts.who_has:
if ts.run_spec:
recommendations[ts.key] = "released"
else: # pure data
recommendations[ts.key] = "forgotten"
self.transitions(recommendations, stimulus_id=stimulus_id)
for plugin in list(self.plugins.values()):
try:
result = plugin.remove_worker(scheduler=self, worker=address)
if inspect.isawaitable(result):
await result
except Exception as e:
logger.exception(e)
if not self.workers:
logger.info("Lost all workers")
for w in self.workers:
self.bandwidth_workers.pop((address, w), None)
self.bandwidth_workers.pop((w, address), None)
async def remove_worker_from_events():
# If the worker isn't registered anymore after the delay, remove from events
if address not in self.workers and address in self.events:
del self.events[address]
cleanup_delay = parse_timedelta(
dask.config.get("distributed.scheduler.events-cleanup-delay")
)
self._ongoing_background_tasks.call_later(
cleanup_delay, remove_worker_from_events
)
logger.debug("Removed worker %s", ws)
return "OK"
[docs] async def stimulus_cancel(self, keys, client, force=False):
"""Stop execution on a list of keys"""
logger.info("Client %s requests to cancel %d keys", client, len(keys))
if client:
self.log_event(
client, {"action": "cancel", "count": len(keys), "force": force}
)
await asyncio.gather(
*[self._cancel_key(key, client, force=force) for key in keys]
)
async def _cancel_key(self, key, client, force=False):
"""Cancel a particular key and all dependents"""
# TODO: this should be converted to use the transition mechanism
ts: TaskState | None = self.tasks.get(key)
dts: TaskState
try:
cs: ClientState = self.clients[client]
except KeyError:
return
# no key yet, lets try again in a moment
start = time()
while ts is None or not ts.who_wants:
await asyncio.sleep(0.1)
ts = self.tasks.get(key)
if time() - start >= 1:
return
if force or ts.who_wants == {cs}: # no one else wants this key
await asyncio.gather(
*[
self._cancel_key(dts.key, client, force=force)
for dts in ts.dependents
]
)
logger.info("Scheduler cancels key %s. Force=%s", key, force)
self.report({"op": "cancelled-key", "key": key})
clients = list(ts.who_wants) if force else [cs]
for cs in clients:
self.client_releases_keys(
keys=[key], client=cs.client_key, stimulus_id=f"cancel-key-{time()}"
)
def client_desires_keys(self, keys=None, client=None):
cs: ClientState = self.clients.get(client)
if cs is None:
# For publish, queues etc.
self.clients[client] = cs = ClientState(client)
for k in keys:
ts = self.tasks.get(k)
if ts is None:
# For publish, queues etc.
ts = self.new_task(k, None, "released")
ts.who_wants.add(cs)
cs.wants_what.add(ts)
if ts.state in ("memory", "erred"):
self.report_on_key(ts=ts, client=client)
[docs] def client_releases_keys(self, keys=None, client=None, stimulus_id=None):
"""Remove keys from client desired list"""
stimulus_id = stimulus_id or f"client-releases-keys-{time()}"
if not isinstance(keys, list):
keys = list(keys)
cs: ClientState = self.clients[client]
recommendations: dict = {}
_client_releases_keys(self, keys=keys, cs=cs, recommendations=recommendations)
self.transitions(recommendations, stimulus_id)
[docs] def client_heartbeat(self, client=None):
"""Handle heartbeats from Client"""
cs: ClientState = self.clients[client]
cs.last_seen = time()
###################
# Task Validation #
###################
def validate_released(self, key):
ts: TaskState = self.tasks[key]
assert ts.state == "released"
assert not ts.waiters
assert not ts.waiting_on
assert not ts.who_has
assert not ts.processing_on
assert not any([ts in dts.waiters for dts in ts.dependencies])
assert ts not in self.unrunnable
def validate_waiting(self, key):
ts: TaskState = self.tasks[key]
assert ts.waiting_on
assert not ts.who_has
assert not ts.processing_on
assert ts not in self.unrunnable
for dts in ts.dependencies:
# We are waiting on a dependency iff it's not stored
assert bool(dts.who_has) != (dts in ts.waiting_on)
assert ts in dts.waiters # XXX even if dts._who_has?
def validate_processing(self, key):
ts: TaskState = self.tasks[key]
dts: TaskState
assert not ts.waiting_on
ws: WorkerState = ts.processing_on
assert ws
assert ts in ws.processing
assert not ts.who_has
for dts in ts.dependencies:
assert dts.who_has
assert ts in dts.waiters
def validate_memory(self, key):
ts: TaskState = self.tasks[key]
dts: TaskState
assert ts.who_has
assert bool(ts in self.replicated_tasks) == (len(ts.who_has) > 1)
assert not ts.processing_on
assert not ts.waiting_on
assert ts not in self.unrunnable
for dts in ts.dependents:
assert (dts in ts.waiters) == (
dts.state in ("waiting", "processing", "no-worker")
)
assert ts not in dts.waiting_on
def validate_no_worker(self, key):
ts: TaskState = self.tasks[key]
assert ts in self.unrunnable
assert not ts.waiting_on
assert ts in self.unrunnable
assert not ts.processing_on
assert not ts.who_has
for dts in ts.dependencies:
assert dts.who_has
def validate_erred(self, key):
ts: TaskState = self.tasks[key]
assert ts.exception_blame
assert not ts.who_has
def validate_key(self, key, ts: TaskState | None = None):
try:
if ts is None:
ts = self.tasks.get(key)
if ts is None:
logger.debug("Key lost: %s", key)
else:
ts.validate()
try:
func = getattr(self, "validate_" + ts.state.replace("-", "_"))
except AttributeError:
logger.error(
"self.validate_%s not found", ts.state.replace("-", "_")
)
else:
func(key)
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def validate_state(self, allow_overlap: bool = False) -> None:
validate_state(self.tasks, self.workers, self.clients)
if not (set(self.workers) == set(self.stream_comms)):
raise ValueError("Workers not the same in all collections")
for w, ws in self.workers.items():
assert isinstance(w, str), (type(w), w)
assert isinstance(ws, WorkerState), (type(ws), ws)
assert ws.address == w
if not ws.processing:
assert not ws.occupancy
assert ws.address in self.idle
assert (ws.status == Status.running) == (ws in self.running)
for ws in self.running:
assert ws.status == Status.running
assert ws.address in self.workers
for k, ts in self.tasks.items():
assert isinstance(ts, TaskState), (type(ts), ts)
assert ts.key == k
assert bool(ts in self.replicated_tasks) == (len(ts.who_has) > 1)
self.validate_key(k, ts)
for ts in self.replicated_tasks:
assert ts.state == "memory"
assert ts.key in self.tasks
for c, cs in self.clients.items():
# client=None is often used in tests...
assert c is None or type(c) == str, (type(c), c)
assert type(cs) == ClientState, (type(cs), cs)
assert cs.client_key == c
a = {w: ws.nbytes for w, ws in self.workers.items()}
b = {
w: sum(ts.get_nbytes() for ts in ws.has_what)
for w, ws in self.workers.items()
}
assert a == b, (a, b)
actual_total_occupancy = 0
for worker, ws in self.workers.items():
ws_processing_total = sum(
cost for ts, cost in ws.processing.items() if ts not in ws.long_running
)
assert abs(ws_processing_total - ws.occupancy) < 1e-8, (
worker,
ws_processing_total,
ws.occupancy,
)
actual_total_occupancy += ws.occupancy
assert abs(actual_total_occupancy - self.total_occupancy) < 1e-8, (
actual_total_occupancy,
self.total_occupancy,
)
if self.transition_counter_max:
assert self.transition_counter < self.transition_counter_max
###################
# Manage Messages #
###################
[docs] def report(self, msg: dict, ts: TaskState | None = None, client: str | None = None):
"""
Publish updates to all listening Queues and Comms
If the message contains a key then we only send the message to those
comms that care about the key.
"""
if ts is None:
msg_key = msg.get("key")
if msg_key is not None:
tasks: dict = self.tasks
ts = tasks.get(msg_key)
client_comms: dict = self.client_comms
if ts is None:
# Notify all clients
client_keys = list(client_comms)
elif client is None:
# Notify clients interested in key
client_keys = [cs.client_key for cs in ts.who_wants]
else:
# Notify clients interested in key (including `client`)
client_keys = [
cs.client_key for cs in ts.who_wants if cs.client_key != client
]
client_keys.append(client)
k: str
for k in client_keys:
c = client_comms.get(k)
if c is None:
continue
try:
c.send(msg)
# logger.debug("Scheduler sends message to client %s", msg)
except CommClosedError:
if self.status == Status.running:
logger.critical(
"Closed comm %r while trying to write %s", c, msg, exc_info=True
)
[docs] async def add_client(
self, comm: Comm, client: str, versions: dict[str, Any]
) -> None:
"""Add client to network
We listen to all future messages from this Comm.
"""
assert client is not None
comm.name = "Scheduler->Client"
logger.info("Receive client connection: %s", client)
self.log_event(["all", client], {"action": "add-client", "client": client})
self.clients[client] = ClientState(client, versions=versions)
for plugin in list(self.plugins.values()):
try:
plugin.add_client(scheduler=self, client=client)
except Exception as e:
logger.exception(e)
try:
bcomm = BatchedSend(interval="2ms", loop=self.loop)
bcomm.start(comm)
self.client_comms[client] = bcomm
msg = {"op": "stream-start"}
version_warning = version_module.error_message(
version_module.get_versions(),
{w: ws.versions for w, ws in self.workers.items()},
versions,
)
msg.update(version_warning)
bcomm.send(msg)
try:
await self.handle_stream(comm=comm, extra={"client": client})
finally:
self.remove_client(client=client, stimulus_id=f"remove-client-{time()}")
logger.debug("Finished handling client %s", client)
finally:
if not comm.closed():
self.client_comms[client].send({"op": "stream-closed"})
try:
if not sys.is_finalizing():
await self.client_comms[client].close()
del self.client_comms[client]
if self.status == Status.running:
logger.info("Close client connection: %s", client)
except TypeError: # comm becomes None during GC
pass
[docs] def remove_client(self, client: str, stimulus_id: str | None = None) -> None:
"""Remove client from network"""
stimulus_id = stimulus_id or f"remove-client-{time()}"
if self.status == Status.running:
logger.info("Remove client %s", client)
self.log_event(["all", client], {"action": "remove-client", "client": client})
try:
cs: ClientState = self.clients[client]
except KeyError:
# XXX is this a legitimate condition?
pass
else:
self.client_releases_keys(
keys=[ts.key for ts in cs.wants_what],
client=cs.client_key,
stimulus_id=stimulus_id,
)
del self.clients[client]
for plugin in list(self.plugins.values()):
try:
plugin.remove_client(scheduler=self, client=client)
except Exception as e:
logger.exception(e)
async def remove_client_from_events():
# If the client isn't registered anymore after the delay, remove from events
if client not in self.clients and client in self.events:
del self.events[client]
cleanup_delay = parse_timedelta(
dask.config.get("distributed.scheduler.events-cleanup-delay")
)
if not self._ongoing_background_tasks.closed:
self._ongoing_background_tasks.call_later(
cleanup_delay, remove_client_from_events
)
[docs] def send_task_to_worker(self, worker, ts: TaskState, duration: float = -1):
"""Send a single computational task to a worker"""
try:
msg: dict = _task_to_msg(self, ts, duration)
self.worker_send(worker, msg)
except Exception as e:
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
raise
def handle_uncaught_error(self, **msg):
logger.exception(clean_exception(**msg)[1])
def handle_task_finished(
self, key: str, worker: str, stimulus_id: str, **msg
) -> None:
if worker not in self.workers:
return
validate_key(key)
r: tuple = self.stimulus_task_finished(
key=key, worker=worker, stimulus_id=stimulus_id, **msg
)
recommendations, client_msgs, worker_msgs = r
self._transitions(recommendations, client_msgs, worker_msgs, stimulus_id)
self.send_all(client_msgs, worker_msgs)
def handle_task_erred(self, key: str, stimulus_id: str, **msg) -> None:
r: tuple = self.stimulus_task_erred(key=key, stimulus_id=stimulus_id, **msg)
recommendations, client_msgs, worker_msgs = r
self._transitions(recommendations, client_msgs, worker_msgs, stimulus_id)
self.send_all(client_msgs, worker_msgs)
def release_worker_data(self, key: str, worker: str, stimulus_id: str) -> None:
ts = self.tasks.get(key)
ws = self.workers.get(worker)
if not ts or not ws or ws not in ts.who_has:
return
self.remove_replica(ts, ws)
if not ts.who_has:
self.transitions({key: "released"}, stimulus_id)
[docs] def handle_long_running(
self, key: str, worker: str, compute_duration: float, stimulus_id: str
) -> None:
"""A task has seceded from the thread pool
We stop the task from being stolen in the future, and change task
duration accounting as if the task has stopped.
"""
if key not in self.tasks:
logger.debug("Skipping long_running since key %s was already released", key)
return
ts = self.tasks[key]
steal = self.extensions.get("stealing")
if steal is not None:
steal.remove_key_from_stealable(ts)
ws = ts.processing_on
if ws is None:
logger.debug("Received long-running signal from duplicate task. Ignoring.")
return
old_duration = ts.prefix.duration_average
if old_duration < 0:
ts.prefix.duration_average = compute_duration
else:
ts.prefix.duration_average = (old_duration + compute_duration) / 2
occ = ws.processing[ts]
ws.occupancy -= occ
self.total_occupancy -= occ
# Cannot remove from processing since we're using this for things like
# idleness detection. Idle workers are typically targeted for
# downscaling but we should not downscale workers with long running
# tasks
ws.processing[ts] = 0
ws.long_running.add(ts)
self.check_idle_saturated(ws)
def handle_worker_status_change(
self, status: str, worker: str, stimulus_id: str
) -> None:
ws = self.workers.get(worker)
if not ws:
return
prev_status = ws.status
ws.status = Status.lookup[status] # type: ignore
if ws.status == prev_status:
return
self.log_event(
ws.address,
{
"action": "worker-status-change",
"prev-status": prev_status.name,
"status": status,
},
)
if ws.status == Status.running:
self.running.add(ws)
recs = self.bulk_schedule_after_adding_worker(ws)
if recs:
client_msgs: dict = {}
worker_msgs: dict = {}
self._transitions(recs, client_msgs, worker_msgs, stimulus_id)
self.send_all(client_msgs, worker_msgs)
else:
self.running.discard(ws)
[docs] async def handle_request_refresh_who_has(
self, keys: Iterable[str], worker: str, stimulus_id: str
) -> None:
"""Asynchronous request (through bulk comms) from a Worker to refresh the
who_has for some keys. Not to be confused with scheduler.who_has, which is a
synchronous RPC request from a Client.
"""
self.stream_comms[worker].send(
{
"op": "refresh-who-has",
"who_has": self.get_who_has(keys),
"stimulus_id": stimulus_id,
},
)
[docs] async def handle_worker(self, comm=None, worker=None, stimulus_id=None):
"""
Listen to responses from a single worker
This is the main loop for scheduler-worker interaction
See Also
--------
Scheduler.handle_client: Equivalent coroutine for clients
"""
assert stimulus_id
comm.name = "Scheduler connection to worker"
worker_comm = self.stream_comms[worker]
worker_comm.start(comm)
logger.info("Starting worker compute stream, %s", worker)
try:
await self.handle_stream(comm=comm, extra={"worker": worker})
finally:
if worker in self.stream_comms:
worker_comm.abort()
await self.remove_worker(address=worker, stimulus_id=stimulus_id)
[docs] def add_plugin(
self,
plugin: SchedulerPlugin,
*,
idempotent: bool = False,
name: str | None = None,
**kwargs,
):
"""Add external plugin to scheduler.
See https://distributed.readthedocs.io/en/latest/plugins.html
Parameters
----------
plugin : SchedulerPlugin
SchedulerPlugin instance to add
idempotent : bool
If true, the plugin is assumed to already exist and no
action is taken.
name : str
A name for the plugin, if None, the name attribute is
checked on the Plugin instance and generated if not
discovered.
"""
if name is None:
name = _get_plugin_name(plugin)
if name in self.plugins:
if idempotent:
return
warnings.warn(
f"Scheduler already contains a plugin with name {name}; overwriting.",
category=UserWarning,
)
self.plugins[name] = plugin
[docs] def remove_plugin(
self,
name: str | None = None,
plugin: SchedulerPlugin | None = None,
) -> None:
"""Remove external plugin from scheduler
Parameters
----------
name : str
Name of the plugin to remove
"""
assert name is not None
try:
del self.plugins[name]
except KeyError:
raise ValueError(
f"Could not find plugin {name!r} among the current scheduler plugins"
)
[docs] async def register_scheduler_plugin(self, plugin, name=None, idempotent=None):
"""Register a plugin on the scheduler."""
if not dask.config.get("distributed.scheduler.pickle"):
raise ValueError(
"Cannot register a scheduler plugin as the scheduler "
"has been explicitly disallowed from deserializing "
"arbitrary bytestrings using pickle via the "
"'distributed.scheduler.pickle' configuration setting."
)
if not isinstance(plugin, SchedulerPlugin):
plugin = loads(plugin)
if name is None:
name = _get_plugin_name(plugin)
if name in self.plugins and idempotent:
return
if hasattr(plugin, "start"):
result = plugin.start(self)
if inspect.isawaitable(result):
await result
self.add_plugin(plugin, name=name, idempotent=idempotent)
[docs] def worker_send(self, worker: str, msg: dict[str, Any]) -> None:
"""Send message to worker
This also handles connection failures by adding a callback to remove
the worker on the next cycle.
"""
stream_comms: dict = self.stream_comms
try:
stream_comms[worker].send(msg)
except (CommClosedError, AttributeError):
self._ongoing_background_tasks.call_soon(
self.remove_worker,
address=worker,
stimulus_id=f"worker-send-comm-fail-{time()}",
)
[docs] def client_send(self, client, msg):
"""Send message to client"""
client_comms: dict = self.client_comms
c = client_comms.get(client)
if c is None:
return
try:
c.send(msg)
except CommClosedError:
if self.status == Status.running:
logger.critical(
"Closed comm %r while trying to write %s", c, msg, exc_info=True
)
[docs] def send_all(self, client_msgs: dict, worker_msgs: dict):
"""Send messages to client and workers"""
client_comms: dict = self.client_comms
stream_comms: dict = self.stream_comms
msgs: list
for client, msgs in client_msgs.items():
c = client_comms.get(client)
if c is None:
continue
try:
c.send(*msgs)
except CommClosedError:
if self.status == Status.running:
logger.critical(
"Closed comm %r while trying to write %s",
c,
msgs,
exc_info=True,
)
for worker, msgs in worker_msgs.items():
try:
w = stream_comms[worker]
w.send(*msgs)
except KeyError:
# worker already gone
pass
except (CommClosedError, AttributeError):
self._ongoing_background_tasks.call_soon(
self.remove_worker,
address=worker,
stimulus_id=f"send-all-comm-fail-{time()}",
)
############################
# Less common interactions #
############################
[docs] async def scatter(
self,
comm=None,
data=None,
workers=None,
client=None,
broadcast=False,
timeout=2,
):
"""Send data out to workers
See also
--------
Scheduler.broadcast:
"""
start = time()
while True:
if workers is None:
wss = self.running
else:
workers = [self.coerce_address(w) for w in workers]
wss = {self.workers[w] for w in workers}
wss = {ws for ws in wss if ws.status == Status.running}
if wss:
break
if time() > start + timeout:
raise TimeoutError("No valid workers found")
await asyncio.sleep(0.1)
nthreads = {ws.address: ws.nthreads for ws in wss}
assert isinstance(data, dict)
keys, who_has, nbytes = await scatter_to_workers(
nthreads, data, rpc=self.rpc, report=False
)
self.update_data(who_has=who_has, nbytes=nbytes, client=client)
if broadcast:
n = len(nthreads) if broadcast is True else broadcast
await self.replicate(keys=keys, workers=workers, n=n)
self.log_event(
[client, "all"], {"action": "scatter", "client": client, "count": len(data)}
)
return keys
[docs] async def gather(self, keys, serializers=None):
"""Collect data from workers to the scheduler"""
stimulus_id = f"gather-{time()}"
keys = list(keys)
who_has = {}
for key in keys:
ts: TaskState = self.tasks.get(key)
if ts is not None:
who_has[key] = [ws.address for ws in ts.who_has]
else:
who_has[key] = []
data, missing_keys, missing_workers = await gather_from_workers(
who_has, rpc=self.rpc, close=False, serializers=serializers
)
if not missing_keys:
result = {"status": "OK", "data": data}
else:
missing_states = [
(self.tasks[key].state if key in self.tasks else None)
for key in missing_keys
]
logger.exception(
"Couldn't gather keys %s state: %s workers: %s",
missing_keys,
missing_states,
missing_workers,
)
result = {"status": "error", "keys": missing_keys}
with log_errors():
# Remove suspicious workers from the scheduler and shut them down.
await asyncio.gather(
*(
self.remove_worker(
address=worker, close=True, stimulus_id=stimulus_id
)
for worker in missing_workers
)
)
for key, workers in missing_keys.items():
logger.exception(
"Shut down workers that don't have promised key: %s, %s",
str(workers),
str(key),
)
self.log_event("all", {"action": "gather", "count": len(keys)})
return result
def clear_task_state(self):
# XXX what about nested state such as ClientState.wants_what
# (see also fire-and-forget...)
logger.info("Clear task state")
for collection in self._task_state_collections:
collection.clear()
[docs] @log_errors
async def restart(self, client=None, timeout=30):
"""Restart all workers. Reset local state."""
stimulus_id = f"restart-{time()}"
n_workers = len(self.workers)
logger.info("Send lost future signal to clients")
for cs in self.clients.values():
self.client_releases_keys(
keys=[ts.key for ts in cs.wants_what],
client=cs.client_key,
stimulus_id=stimulus_id,
)
nanny_workers = {
addr: ws.nanny for addr, ws in self.workers.items() if ws.nanny
}
# Close non-Nanny workers. We have no way to restart them, so we just let them go,
# and assume a deployment system is going to restart them for us.
await asyncio.gather(
*(
self.remove_worker(address=addr, stimulus_id=stimulus_id)
for addr in self.workers
if addr not in nanny_workers
)
)
self.clear_task_state()
for plugin in list(self.plugins.values()):
try:
plugin.restart(self)
except Exception as e:
logger.exception(e)
logger.debug("Send kill signal to nannies: %s", nanny_workers)
async with contextlib.AsyncExitStack() as stack:
nannies = [
await stack.enter_async_context(
rpc(nanny_address, connection_args=self.connection_args)
)
for nanny_address in nanny_workers.values()
]
try:
resps = await asyncio.wait_for(
asyncio.gather(
*(
nanny.restart(close=True, timeout=timeout * 0.8)
for nanny in nannies
)
),
timeout,
)
# NOTE: the `WorkerState` entries for these workers will be removed
# naturally when they disconnect from the scheduler.
except TimeoutError:
logger.error(
"Nannies didn't report back restarted within "
"timeout. Continuing with restart process"
)
else:
if not all(resp == "OK" for resp in resps):
logger.error(
"Not all workers responded positively: %s",
resps,
exc_info=True,
)
self.clear_task_state()
with suppress(AttributeError):
for c in self._worker_coroutines:
c.cancel()
self.log_event([client, "all"], {"action": "restart", "client": client})
start = time()
while time() < start + 10 and len(self.workers) < n_workers:
await asyncio.sleep(0.01)
self.report({"op": "restart"})
[docs] async def broadcast(
self,
comm=None,
*,
msg: dict,
workers: "list[str] | None" = None,
hosts: "list[str] | None" = None,
nanny: bool = False,
serializers=None,
on_error: "Literal['raise', 'return', 'return_pickle', 'ignore']" = "raise",
) -> dict: # dict[str, Any]
"""Broadcast message to workers, return all results"""
if workers is None:
if hosts is None:
workers = list(self.workers)
else:
workers = []
if hosts is not None:
for host in hosts:
dh: dict = self.host_info.get(host) # type: ignore
if dh is not None:
workers.extend(dh["addresses"])
# TODO replace with worker_list
if nanny:
addresses = [self.workers[w].nanny for w in workers]
else:
addresses = workers
ERROR = object()
async def send_message(addr):
try:
comm = await self.rpc.connect(addr)
comm.name = "Scheduler Broadcast"
try:
resp = await send_recv(
comm, close=True, serializers=serializers, **msg
)
finally:
self.rpc.reuse(addr, comm)
return resp
except Exception as e:
logger.error(f"broadcast to {addr} failed: {e.__class__.__name__}: {e}")
if on_error == "raise":
raise
elif on_error == "return":
return e
elif on_error == "return_pickle":
return dumps(e, protocol=4)
elif on_error == "ignore":
return ERROR
else:
raise ValueError(
"on_error must be 'raise', 'return', 'return_pickle', "
f"or 'ignore'; got {on_error!r}"
)
results = await All(
[send_message(address) for address in addresses if address is not None]
)
return {k: v for k, v in zip(workers, results) if v is not ERROR}
[docs] async def proxy(self, comm=None, msg=None, worker=None, serializers=None):
"""Proxy a communication through the scheduler to some other worker"""
d = await self.broadcast(
comm=comm, msg=msg, workers=[worker], serializers=serializers
)
return d[worker]
[docs] async def gather_on_worker(
self, worker_address: str, who_has: "dict[str, list[str]]"
) -> set:
"""Peer-to-peer copy of keys from multiple workers to a single worker
Parameters
----------
worker_address: str
Recipient worker address to copy keys to
who_has: dict[Hashable, list[str]]
{key: [sender address, sender address, ...], key: ...}
Returns
-------
returns:
set of keys that failed to be copied
"""
try:
result = await retry_operation(
self.rpc(addr=worker_address).gather, who_has=who_has
)
except OSError as e:
# This can happen e.g. if the worker is going through controlled shutdown;
# it doesn't necessarily mean that it went unexpectedly missing
logger.warning(
f"Communication with worker {worker_address} failed during "
f"replication: {e.__class__.__name__}: {e}"
)
return set(who_has)
ws = self.workers.get(worker_address)
if not ws:
logger.warning(f"Worker {worker_address} lost during replication")
return set(who_has)
elif result["status"] == "OK":
keys_failed = set()
keys_ok: Set = who_has.keys()
elif result["status"] == "partial-fail":
keys_failed = set(result["keys"])
keys_ok = who_has.keys() - keys_failed
logger.warning(
f"Worker {worker_address} failed to acquire keys: {result['keys']}"
)
else: # pragma: nocover
raise ValueError(f"Unexpected message from {worker_address}: {result}")
for key in keys_ok:
ts: TaskState = self.tasks.get(key) # type: ignore
if ts is None or ts.state != "memory":
logger.warning(f"Key lost during replication: {key}")
continue
if ws not in ts.who_has:
self.add_replica(ts, ws)
return keys_failed
[docs] async def delete_worker_data(
self, worker_address: str, keys: "Collection[str]", stimulus_id: str
) -> None:
"""Delete data from a worker and update the corresponding worker/task states
Parameters
----------
worker_address: str
Worker address to delete keys from
keys: list[str]
List of keys to delete on the specified worker
"""
try:
await retry_operation(
self.rpc(addr=worker_address).free_keys,
keys=list(keys),
stimulus_id=f"delete-data-{time()}",
)
except OSError as e:
# This can happen e.g. if the worker is going through controlled shutdown;
# it doesn't necessarily mean that it went unexpectedly missing
logger.warning(
f"Communication with worker {worker_address} failed during "
f"replication: {e.__class__.__name__}: {e}"
)
return
ws = self.workers.get(worker_address)
if not ws:
return
for key in keys:
ts: TaskState = self.tasks.get(key) # type: ignore
if ts is not None and ws in ts.who_has:
assert ts.state == "memory"
self.remove_replica(ts, ws)
if not ts.who_has:
# Last copy deleted
self.transitions({key: "released"}, stimulus_id)
self.log_event(ws.address, {"action": "remove-worker-data", "keys": keys})
[docs] @log_errors
async def rebalance(
self,
comm=None,
keys: "Iterable[Hashable]" | None = None,
workers: "Iterable[str]" | None = None,
stimulus_id: str | None = None,
) -> dict:
"""Rebalance keys so that each worker ends up with roughly the same process
memory (managed+unmanaged).
.. warning::
This operation is generally not well tested against normal operation of the
scheduler. It is not recommended to use it while waiting on computations.
**Algorithm**
#. Find the mean occupancy of the cluster, defined as data managed by dask +
unmanaged process memory that has been there for at least 30 seconds
(``distributed.worker.memory.recent-to-old-time``).
This lets us ignore temporary spikes caused by task heap usage.
Alternatively, you may change how memory is measured both for the individual
workers as well as to calculate the mean through
``distributed.worker.memory.rebalance.measure``. Namely, this can be useful
to disregard inaccurate OS memory measurements.
#. Discard workers whose occupancy is within 5% of the mean cluster occupancy
(``distributed.worker.memory.rebalance.sender-recipient-gap`` / 2).
This helps avoid data from bouncing around the cluster repeatedly.
#. Workers above the mean are senders; those below are recipients.
#. Discard senders whose absolute occupancy is below 30%
(``distributed.worker.memory.rebalance.sender-min``). In other words, no data
is moved regardless of imbalancing as long as all workers are below 30%.
#. Discard recipients whose absolute occupancy is above 60%
(``distributed.worker.memory.rebalance.recipient-max``).
Note that this threshold by default is the same as
``distributed.worker.memory.target`` to prevent workers from accepting data
and immediately spilling it out to disk.
#. Iteratively pick the sender and recipient that are farthest from the mean and
move the *least recently inserted* key between the two, until either all
senders or all recipients fall within 5% of the mean.
A recipient will be skipped if it already has a copy of the data. In other
words, this method does not degrade replication.
A key will be skipped if there are no recipients available with enough memory
to accept the key and that don't already hold a copy.
The least recently insertd (LRI) policy is a greedy choice with the advantage of
being O(1), trivial to implement (it relies on python dict insertion-sorting)
and hopefully good enough in most cases. Discarded alternative policies were:
- Largest first. O(n*log(n)) save for non-trivial additional data structures and
risks causing the largest chunks of data to repeatedly move around the
cluster like pinballs.
- Least recently used (LRU). This information is currently available on the
workers only and not trivial to replicate on the scheduler; transmitting it
over the network would be very expensive. Also, note that dask will go out of
its way to minimise the amount of time intermediate keys are held in memory,
so in such a case LRI is a close approximation of LRU.
Parameters
----------
keys: optional
allowlist of dask keys that should be considered for moving. All other keys
will be ignored. Note that this offers no guarantee that a key will actually
be moved (e.g. because it is unnecessary or because there are no viable
recipient workers for it).
workers: optional
allowlist of workers addresses to be considered as senders or recipients.
All other workers will be ignored. The mean cluster occupancy will be
calculated only using the allowed workers.
"""
stimulus_id = stimulus_id or f"rebalance-{time()}"
if workers is not None:
wss = [self.workers[w] for w in workers]
else:
wss = self.workers.values()
if not wss:
return {"status": "OK"}
if keys is not None:
if not isinstance(keys, Set):
keys = set(keys) # unless already a set-like
if not keys:
return {"status": "OK"}
missing_data = [
k for k in keys if k not in self.tasks or not self.tasks[k].who_has
]
if missing_data:
return {"status": "partial-fail", "keys": missing_data}
msgs = self._rebalance_find_msgs(keys, wss)
if not msgs:
return {"status": "OK"}
async with self._lock:
result = await self._rebalance_move_data(msgs, stimulus_id)
if result["status"] == "partial-fail" and keys is None:
# Only return failed keys if the client explicitly asked for them
result = {"status": "OK"}
return result
def _rebalance_find_msgs(
self,
keys: Set[Hashable] | None,
workers: Iterable[WorkerState],
) -> list[tuple[WorkerState, WorkerState, TaskState]]:
"""Identify workers that need to lose keys and those that can receive them,
together with how many bytes each needs to lose/receive. Then, pair a sender
worker with a recipient worker for each key, until the cluster is rebalanced.
This method only defines the work to be performed; it does not start any network
transfers itself.
The big-O complexity is O(wt + ke*log(we)), where
- wt is the total number of workers on the cluster (or the number of allowed
workers, if explicitly stated by the user)
- we is the number of workers that are eligible to be senders or recipients
- kt is the total number of keys on the cluster (or on the allowed workers)
- ke is the number of keys that need to be moved in order to achieve a balanced
cluster
There is a degenerate edge case O(wt + kt*log(we)) when kt is much greater than
the number of allowed keys, or when most keys are replicated or cannot be
moved for some other reason.
Returns list of tuples to feed into _rebalance_move_data:
- sender worker
- recipient worker
- task to be transferred
"""
# Heaps of workers, managed by the heapq module, that need to send/receive data,
# with how many bytes each needs to send/receive.
#
# Each element of the heap is a tuple constructed as follows:
# - snd_bytes_max/rec_bytes_max: maximum number of bytes to send or receive.
# This number is negative, so that the workers farthest from the cluster mean
# are at the top of the smallest-first heaps.
# - snd_bytes_min/rec_bytes_min: minimum number of bytes after sending/receiving
# which the worker should not be considered anymore. This is also negative.
# - arbitrary unique number, there just to to make sure that WorkerState objects
# are never used for sorting in the unlikely event that two processes have
# exactly the same number of bytes allocated.
# - WorkerState
# - iterator of all tasks in memory on the worker (senders only), insertion
# sorted (least recently inserted first).
# Note that this iterator will typically *not* be exhausted. It will only be
# exhausted if, after moving away from the worker all keys that can be moved,
# is insufficient to drop snd_bytes_min above 0.
senders: list[tuple[int, int, int, WorkerState, Iterator[TaskState]]] = []
recipients: list[tuple[int, int, int, WorkerState]] = []
# Output: [(sender, recipient, task), ...]
msgs: list[tuple[WorkerState, WorkerState, TaskState]] = []
# By default, this is the optimistic memory, meaning total process memory minus
# unmanaged memory that appeared over the last 30 seconds
# (distributed.worker.memory.recent-to-old-time).
# This lets us ignore temporary spikes caused by task heap usage.
memory_by_worker = [
(ws, getattr(ws.memory, self.MEMORY_REBALANCE_MEASURE)) for ws in workers
]
mean_memory = sum(m for _, m in memory_by_worker) // len(memory_by_worker)
for ws, ws_memory in memory_by_worker:
if ws.memory_limit:
half_gap = int(self.MEMORY_REBALANCE_HALF_GAP * ws.memory_limit)
sender_min = self.MEMORY_REBALANCE_SENDER_MIN * ws.memory_limit
recipient_max = self.MEMORY_REBALANCE_RECIPIENT_MAX * ws.memory_limit
else:
half_gap = 0
sender_min = 0.0
recipient_max = math.inf
if (
ws._has_what
and ws_memory >= mean_memory + half_gap
and ws_memory >= sender_min
):
# This may send the worker below sender_min (by design)
snd_bytes_max = mean_memory - ws_memory # negative
snd_bytes_min = snd_bytes_max + half_gap # negative
# See definition of senders above
senders.append(
(snd_bytes_max, snd_bytes_min, id(ws), ws, iter(ws._has_what))
)
elif ws_memory < mean_memory - half_gap and ws_memory < recipient_max:
# This may send the worker above recipient_max (by design)
rec_bytes_max = ws_memory - mean_memory # negative
rec_bytes_min = rec_bytes_max + half_gap # negative
# See definition of recipients above
recipients.append((rec_bytes_max, rec_bytes_min, id(ws), ws))
# Fast exit in case no transfers are necessary or possible
if not senders or not recipients:
self.log_event(
"all",
{
"action": "rebalance",
"senders": len(senders),
"recipients": len(recipients),
"moved_keys": 0,
},
)
return []
heapq.heapify(senders)
heapq.heapify(recipients)
while senders and recipients:
snd_bytes_max, snd_bytes_min, _, snd_ws, ts_iter = senders[0]
# Iterate through tasks in memory, least recently inserted first
for ts in ts_iter:
if keys is not None and ts.key not in keys:
continue
nbytes = ts.nbytes
if nbytes + snd_bytes_max > 0:
# Moving this task would cause the sender to go below mean and
# potentially risk becoming a recipient, which would cause tasks to
# bounce around. Move on to the next task of the same sender.
continue
# Find the recipient, farthest from the mean, which
# 1. has enough available RAM for this task, and
# 2. doesn't hold a copy of this task already
# There may not be any that satisfies these conditions; in this case
# this task won't be moved.
skipped_recipients = []
use_recipient = False
while recipients and not use_recipient:
rec_bytes_max, rec_bytes_min, _, rec_ws = recipients[0]
if nbytes + rec_bytes_max > 0:
# recipients are sorted by rec_bytes_max.
# The next ones will be worse; no reason to continue iterating
break
use_recipient = ts not in rec_ws._has_what
if not use_recipient:
skipped_recipients.append(heapq.heappop(recipients))
for recipient in skipped_recipients:
heapq.heappush(recipients, recipient)
if not use_recipient:
# This task has no recipients available. Leave it on the sender and
# move on to the next task of the same sender.
continue
# Schedule task for transfer from sender to recipient
msgs.append((snd_ws, rec_ws, ts))
# *_bytes_max/min are all negative for heap sorting
snd_bytes_max += nbytes
snd_bytes_min += nbytes
rec_bytes_max += nbytes
rec_bytes_min += nbytes
# Stop iterating on the tasks of this sender for now and, if it still
# has bytes to lose, push it back into the senders heap; it may or may
# not come back on top again.
if snd_bytes_min < 0:
# See definition of senders above
heapq.heapreplace(
senders,
(snd_bytes_max, snd_bytes_min, id(snd_ws), snd_ws, ts_iter),
)
else:
heapq.heappop(senders)
# If recipient still has bytes to gain, push it back into the recipients
# heap; it may or may not come back on top again.
if rec_bytes_min < 0:
# See definition of recipients above
heapq.heapreplace(
recipients,
(rec_bytes_max, rec_bytes_min, id(rec_ws), rec_ws),
)
else:
heapq.heappop(recipients)
# Move to next sender with the most data to lose.
# It may or may not be the same sender again.
break
else: # for ts in ts_iter
# Exhausted tasks on this sender
heapq.heappop(senders)
return msgs
async def _rebalance_move_data(
self, msgs: "list[tuple[WorkerState, WorkerState, TaskState]]", stimulus_id: str
) -> dict:
"""Perform the actual transfer of data across the network in rebalance().
Takes in input the output of _rebalance_find_msgs(), that is a list of tuples:
- sender worker
- recipient worker
- task to be transferred
FIXME this method is not robust when the cluster is not idle.
"""
to_recipients: defaultdict[str, defaultdict[str, list[str]]] = defaultdict(
lambda: defaultdict(list)
)
for snd_ws, rec_ws, ts in msgs:
to_recipients[rec_ws.address][ts.key].append(snd_ws.address)
failed_keys_by_recipient = dict(
zip(
to_recipients,
await asyncio.gather(
*(
# Note: this never raises exceptions
self.gather_on_worker(w, who_has)
for w, who_has in to_recipients.items()
)
),
)
)
to_senders = defaultdict(list)
for snd_ws, rec_ws, ts in msgs:
if ts.key not in failed_keys_by_recipient[rec_ws.address]:
to_senders[snd_ws.address].append(ts.key)
# Note: this never raises exceptions
await asyncio.gather(
*(self.delete_worker_data(r, v, stimulus_id) for r, v in to_senders.items())
)
for r, v in to_recipients.items():
self.log_event(r, {"action": "rebalance", "who_has": v})
self.log_event(
"all",
{
"action": "rebalance",
"senders": valmap(len, to_senders),
"recipients": valmap(len, to_recipients),
"moved_keys": len(msgs),
},
)
missing_keys = {k for r in failed_keys_by_recipient.values() for k in r}
if missing_keys:
return {"status": "partial-fail", "keys": list(missing_keys)}
else:
return {"status": "OK"}
[docs] async def replicate(
self,
comm=None,
keys=None,
n=None,
workers=None,
branching_factor=2,
delete=True,
lock=True,
stimulus_id=None,
):
"""Replicate data throughout cluster
This performs a tree copy of the data throughout the network
individually on each piece of data.
Parameters
----------
keys: Iterable
list of keys to replicate
n: int
Number of replications we expect to see within the cluster
branching_factor: int, optional
The number of workers that can copy data in each generation.
The larger the branching factor, the more data we copy in
a single step, but the more a given worker risks being
swamped by data requests.
See also
--------
Scheduler.rebalance
"""
stimulus_id = stimulus_id or f"replicate-{time()}"
assert branching_factor > 0
async with self._lock if lock else empty_context:
if workers is not None:
workers = {self.workers[w] for w in self.workers_list(workers)}
workers = {ws for ws in workers if ws.status == Status.running}
else:
workers = self.running
if n is None:
n = len(workers)
else:
n = min(n, len(workers))
if n == 0:
raise ValueError("Can not use replicate to delete data")
tasks = {self.tasks[k] for k in keys}
missing_data = [ts.key for ts in tasks if not ts.who_has]
if missing_data:
return {"status": "partial-fail", "keys": missing_data}
# Delete extraneous data
if delete:
del_worker_tasks = defaultdict(set)
for ts in tasks:
del_candidates = tuple(ts.who_has & workers)
if len(del_candidates) > n:
for ws in random.sample(
del_candidates, len(del_candidates) - n
):
del_worker_tasks[ws].add(ts)
# Note: this never raises exceptions
await asyncio.gather(
*[
self.delete_worker_data(
ws.address, [t.key for t in tasks], stimulus_id
)
for ws, tasks in del_worker_tasks.items()
]
)
# Copy not-yet-filled data
while tasks:
gathers = defaultdict(dict)
for ts in list(tasks):
if ts.state == "forgotten":
# task is no longer needed by any client or dependant task
tasks.remove(ts)
continue
n_missing = n - len(ts.who_has & workers)
if n_missing <= 0:
# Already replicated enough
tasks.remove(ts)
continue
count = min(n_missing, branching_factor * len(ts.who_has))
assert count > 0
for ws in random.sample(tuple(workers - ts.who_has), count):
gathers[ws.address][ts.key] = [
wws.address for wws in ts.who_has
]
await asyncio.gather(
*(
# Note: this never raises exceptions
self.gather_on_worker(w, who_has)
for w, who_has in gathers.items()
)
)
for r, v in gathers.items():
self.log_event(r, {"action": "replicate-add", "who_has": v})
self.log_event(
"all",
{
"action": "replicate",
"workers": list(workers),
"key-count": len(keys),
"branching-factor": branching_factor,
},
)
[docs] def workers_to_close(
self,
comm=None,
memory_ratio: int | float | None = None,
n: int | None = None,
key: Callable[[WorkerState], Hashable] | bytes | None = None,
minimum: int | None = None,
target: int | None = None,
attribute: str = "address",
) -> list[str]:
"""
Find workers that we can close with low cost
This returns a list of workers that are good candidates to retire.
These workers are not running anything and are storing
relatively little data relative to their peers. If all workers are
idle then we still maintain enough workers to have enough RAM to store
our data, with a comfortable buffer.
This is for use with systems like ``distributed.deploy.adaptive``.
Parameters
----------
memory_ratio : Number
Amount of extra space we want to have for our stored data.
Defaults to 2, or that we want to have twice as much memory as we
currently have data.
n : int
Number of workers to close
minimum : int
Minimum number of workers to keep around
key : Callable(WorkerState)
An optional callable mapping a WorkerState object to a group
affiliation. Groups will be closed together. This is useful when
closing workers must be done collectively, such as by hostname.
target : int
Target number of workers to have after we close
attribute : str
The attribute of the WorkerState object to return, like "address"
or "name". Defaults to "address".
Examples
--------
>>> scheduler.workers_to_close()
['tcp://192.168.0.1:1234', 'tcp://192.168.0.2:1234']
Group workers by hostname prior to closing
>>> scheduler.workers_to_close(key=lambda ws: ws.host)
['tcp://192.168.0.1:1234', 'tcp://192.168.0.1:4567']
Remove two workers
>>> scheduler.workers_to_close(n=2)
Keep enough workers to have twice as much memory as we we need.
>>> scheduler.workers_to_close(memory_ratio=2)
Returns
-------
to_close: list of worker addresses that are OK to close
See Also
--------
Scheduler.retire_workers
"""
if target is not None and n is None:
n = len(self.workers) - target
if n is not None:
if n < 0:
n = 0
target = len(self.workers) - n
if n is None and memory_ratio is None:
memory_ratio = 2
with log_errors():
if not n and all([ws.processing for ws in self.workers.values()]):
return []
if key is None:
key = operator.attrgetter("address")
if isinstance(key, bytes) and dask.config.get(
"distributed.scheduler.pickle"
):
key = pickle.loads(key)
groups = groupby(key, self.workers.values())
limit_bytes = {
k: sum(ws.memory_limit for ws in v) for k, v in groups.items()
}
group_bytes = {k: sum(ws.nbytes for ws in v) for k, v in groups.items()}
limit = sum(limit_bytes.values())
total = sum(group_bytes.values())
def _key(group):
is_idle = not any([wws.processing for wws in groups[group]])
bytes = -group_bytes[group]
return is_idle, bytes
idle = sorted(groups, key=_key)
to_close = []
n_remain = len(self.workers)
while idle:
group = idle.pop()
if n is None and any([ws.processing for ws in groups[group]]):
break
if minimum and n_remain - len(groups[group]) < minimum:
break
limit -= limit_bytes[group]
if (
n is not None and n_remain - len(groups[group]) >= (target or 0)
) or (memory_ratio is not None and limit >= memory_ratio * total):
to_close.append(group)
n_remain -= len(groups[group])
else:
break
result = [getattr(ws, attribute) for g in to_close for ws in groups[g]]
if result:
logger.debug("Suggest closing workers: %s", result)
return result
[docs] @log_errors
async def retire_workers(
self,
comm=None,
*,
workers: "list[str] | None" = None,
names: "list | None" = None,
close_workers: bool = False,
remove: bool = True,
stimulus_id: str | None = None,
**kwargs,
) -> dict:
"""Gracefully retire workers from cluster
Parameters
----------
workers: list[str] (optional)
List of worker addresses to retire.
names: list (optional)
List of worker names to retire.
Mutually exclusive with ``workers``.
If neither ``workers`` nor ``names`` are provided, we call
``workers_to_close`` which finds a good set.
close_workers: bool (defaults to False)
Whether or not to actually close the worker explicitly from here.
Otherwise we expect some external job scheduler to finish off the
worker.
remove: bool (defaults to True)
Whether or not to remove the worker metadata immediately or else
wait for the worker to contact us
**kwargs: dict
Extra options to pass to workers_to_close to determine which
workers we should drop
Returns
-------
Dictionary mapping worker ID/address to dictionary of information about
that worker for each retired worker.
See Also
--------
Scheduler.workers_to_close
"""
stimulus_id = stimulus_id or f"retire-workers-{time()}"
# This lock makes retire_workers, rebalance, and replicate mutually
# exclusive and will no longer be necessary once rebalance and replicate are
# migrated to the Active Memory Manager.
# Note that, incidentally, it also prevents multiple calls to retire_workers
# from running in parallel - this is unnecessary.
async with self._lock:
if names is not None:
if workers is not None:
raise TypeError("names and workers are mutually exclusive")
if names:
logger.info("Retire worker names %s", names)
# Support cases where names are passed through a CLI and become
# strings
names_set = {str(name) for name in names}
wss = {ws for ws in self.workers.values() if str(ws.name) in names_set}
elif workers is not None:
wss = {
self.workers[address]
for address in workers
if address in self.workers
}
else:
wss = {
self.workers[address] for address in self.workers_to_close(**kwargs)
}
if not wss:
return {}
stop_amm = False
amm: ActiveMemoryManagerExtension = self.extensions["amm"]
if not amm.running:
amm = ActiveMemoryManagerExtension(
self, policies=set(), register=False, start=True, interval=2.0
)
stop_amm = True
try:
coros = []
for ws in wss:
logger.info("Retiring worker %s", ws.address)
policy = RetireWorker(ws.address)
amm.add_policy(policy)
# Change Worker.status to closing_gracefully. Immediately set
# the same on the scheduler to prevent race conditions.
prev_status = ws.status
ws.status = Status.closing_gracefully
self.running.discard(ws)
# FIXME: We should send a message to the nanny first;
# eventually workers won't be able to close their own nannies.
self.stream_comms[ws.address].send(
{
"op": "worker-status-change",
"status": ws.status.name,
"stimulus_id": stimulus_id,
}
)
coros.append(
self._track_retire_worker(
ws,
policy,
prev_status=prev_status,
close_workers=close_workers,
remove=remove,
stimulus_id=stimulus_id,
)
)
# Give the AMM a kick, in addition to its periodic running. This is
# to avoid unnecessarily waiting for a potentially arbitrarily long
# time (depending on interval settings)
amm.run_once()
workers_info = dict(await asyncio.gather(*coros))
workers_info.pop(None, None)
finally:
if stop_amm:
amm.stop()
self.log_event("all", {"action": "retire-workers", "workers": workers_info})
self.log_event(list(workers_info), {"action": "retired"})
return workers_info
async def _track_retire_worker(
self,
ws: WorkerState,
policy: RetireWorker,
prev_status: Status,
close_workers: bool,
remove: bool,
stimulus_id: str,
) -> tuple: # tuple[str | None, dict]
while not policy.done():
# Sleep 0.01s when there are 4 tasks or less
# Sleep 0.5s when there are 200 or more
poll_interval = max(0.01, min(0.5, len(ws.has_what) / 400))
await asyncio.sleep(poll_interval)
if policy.no_recipients:
# Abort retirement. This time we don't need to worry about race
# conditions and we can wait for a scheduler->worker->scheduler
# round-trip.
self.stream_comms[ws.address].send(
{
"op": "worker-status-change",
"status": prev_status.name,
"stimulus_id": stimulus_id,
}
)
return None, {}
logger.debug(
"All unique keys on worker %s have been replicated elsewhere", ws.address
)
if close_workers and ws.address in self.workers:
await self.close_worker(
worker=ws.address, safe=True, stimulus_id=stimulus_id
)
if remove:
await self.remove_worker(
address=ws.address, safe=True, stimulus_id=stimulus_id
)
logger.info("Retired worker %s", ws.address)
return ws.address, ws.identity()
[docs] def add_keys(self, worker=None, keys=(), stimulus_id=None):
"""
Learn that a worker has certain keys
This should not be used in practice and is mostly here for legacy
reasons. However, it is sent by workers from time to time.
"""
if worker not in self.workers:
return "not found"
ws: WorkerState = self.workers[worker]
redundant_replicas = []
for key in keys:
ts: TaskState = self.tasks.get(key)
if ts is not None and ts.state == "memory":
if ws not in ts.who_has:
self.add_replica(ts, ws)
else:
redundant_replicas.append(key)
if redundant_replicas:
if not stimulus_id:
stimulus_id = f"redundant-replicas-{time()}"
self.worker_send(
worker,
{
"op": "remove-replicas",
"keys": redundant_replicas,
"stimulus_id": stimulus_id,
},
)
return "OK"
[docs] @log_errors
def update_data(
self,
*,
who_has: dict,
nbytes: dict,
client=None,
):
"""
Learn that new data has entered the network from an external source
See Also
--------
Scheduler.mark_key_in_memory
"""
who_has = {k: [self.coerce_address(vv) for vv in v] for k, v in who_has.items()}
logger.debug("Update data %s", who_has)
for key, workers in who_has.items():
ts = self.tasks.get(key)
if ts is None:
ts = self.new_task(key, None, "memory")
ts.state = "memory"
ts_nbytes = nbytes.get(key, -1)
if ts_nbytes >= 0:
ts.set_nbytes(ts_nbytes)
for w in workers:
ws = self.workers[w]
if ws not in ts.who_has:
self.add_replica(ts, ws)
self.report({"op": "key-in-memory", "key": key, "workers": list(workers)})
if client:
self.client_desires_keys(keys=list(who_has), client=client)
def report_on_key(
self,
key: str | None = None,
ts: TaskState | None = None,
client: str | None = None,
):
if ts is None:
ts = self.tasks.get(key)
elif key is None:
key = ts.key
else:
assert False, (key, ts)
if ts is not None:
report_msg = _task_to_report_msg(ts)
else:
report_msg = {"op": "cancelled-key", "key": key}
if report_msg is not None:
self.report(report_msg, ts=ts, client=client)
[docs] @log_errors
async def feed(
self, comm, function=None, setup=None, teardown=None, interval="1s", **kwargs
):
"""
Provides a data Comm to external requester
Caution: this runs arbitrary Python code on the scheduler. This should
eventually be phased out. It is mostly used by diagnostics.
"""
if not dask.config.get("distributed.scheduler.pickle"):
logger.warning(
"Tried to call 'feed' route with custom functions, but "
"pickle is disallowed. Set the 'distributed.scheduler.pickle'"
"config value to True to use the 'feed' route (this is mostly "
"commonly used with progress bars)"
)
return
interval = parse_timedelta(interval)
if function:
function = pickle.loads(function)
if setup:
setup = pickle.loads(setup)
if teardown:
teardown = pickle.loads(teardown)
state = setup(self) if setup else None
if inspect.isawaitable(state):
state = await state
try:
while self.status == Status.running:
if state is None:
response = function(self)
else:
response = function(self, state)
await comm.write(response)
await asyncio.sleep(interval)
except OSError:
pass
finally:
if teardown:
teardown(self, state)
def log_worker_event(
self, worker: str, topic: str | Collection[str], msg: Any
) -> None:
if isinstance(msg, dict):
msg["worker"] = worker
self.log_event(topic, msg)
def subscribe_worker_status(self, comm=None):
WorkerStatusPlugin(self, comm)
ident = self.identity()
for v in ident["workers"].values():
del v["metrics"]
del v["last_seen"]
return ident
def get_processing(self, workers=None):
if workers is not None:
workers = set(map(self.coerce_address, workers))
return {w: [ts.key for ts in self.workers[w].processing] for w in workers}
else:
return {
w: [ts.key for ts in ws.processing] for w, ws in self.workers.items()
}
def get_who_has(self, keys: Iterable[str] | None = None) -> dict[str, list[str]]:
if keys is not None:
return {
key: [ws.address for ws in self.tasks[key].who_has]
if key in self.tasks
else []
for key in keys
}
else:
return {
key: [ws.address for ws in ts.who_has] for key, ts in self.tasks.items()
}
def get_has_what(self, workers=None):
if workers is not None:
workers = map(self.coerce_address, workers)
return {
w: [ts.key for ts in self.workers[w].has_what]
if w in self.workers
else []
for w in workers
}
else:
return {w: [ts.key for ts in ws.has_what] for w, ws in self.workers.items()}
def get_ncores(self, workers=None):
if workers is not None:
workers = map(self.coerce_address, workers)
return {w: self.workers[w].nthreads for w in workers if w in self.workers}
else:
return {w: ws.nthreads for w, ws in self.workers.items()}
def get_ncores_running(self, workers=None):
ncores = self.get_ncores(workers=workers)
return {
w: n for w, n in ncores.items() if self.workers[w].status == Status.running
}
async def get_call_stack(self, keys=None):
if keys is not None:
stack = list(keys)
processing = set()
while stack:
key = stack.pop()
ts = self.tasks[key]
if ts.state == "waiting":
stack.extend([dts.key for dts in ts.dependencies])
elif ts.state == "processing":
processing.add(ts)
workers = defaultdict(list)
for ts in processing:
if ts.processing_on:
workers[ts.processing_on.address].append(ts.key)
else:
workers = {w: None for w in self.workers}
if not workers:
return {}
results = await asyncio.gather(
*(self.rpc(w).call_stack(keys=v) for w, v in workers.items())
)
response = {w: r for w, r in zip(workers, results) if r}
return response
[docs] async def benchmark_hardware(self) -> "dict[str, dict[str, float]]":
"""
Run a benchmark on the workers for memory, disk, and network bandwidths
Returns
-------
result: dict
A dictionary mapping the names "disk", "memory", and "network" to
dictionaries mapping sizes to bandwidths. These bandwidths are
averaged over many workers running computations across the cluster.
"""
out: dict[str, defaultdict[str, list[float]]] = {
name: defaultdict(list) for name in ["disk", "memory", "network"]
}
# disk
result = await self.broadcast(msg={"op": "benchmark_disk"})
for d in result.values():
for size, duration in d.items():
out["disk"][size].append(duration)
# memory
result = await self.broadcast(msg={"op": "benchmark_memory"})
for d in result.values():
for size, duration in d.items():
out["memory"][size].append(duration)
# network
workers = list(self.workers)
# On an adaptive cluster, if multiple workers are started on the same physical host,
# they are more likely to connect to the Scheduler in sequence, ending up next to
# each other in this list.
# The transfer speed within such clusters of workers will be effectively that of
# localhost. This could happen across different VMs and/or docker images, so
# implementing logic based on IP addresses would not necessarily help.
# Randomize the connections to even out the mean measures.
random.shuffle(workers)
futures = [
self.rpc(a).benchmark_network(address=b) for a, b in partition(2, workers)
]
responses = await asyncio.gather(*futures)
for d in responses:
for size, duration in d.items():
out["network"][size].append(duration)
result = {}
for mode in out:
result[mode] = {
size: sum(durations) / len(durations)
for size, durations in out[mode].items()
}
return result
@log_errors
def get_nbytes(self, keys=None, summary=True):
if keys is not None:
result = {k: self.tasks[k].nbytes for k in keys}
else:
result = {k: ts.nbytes for k, ts in self.tasks.items() if ts.nbytes >= 0}
if summary:
out = defaultdict(lambda: 0)
for k, v in result.items():
out[key_split(k)] += v
result = dict(out)
return result
[docs] def run_function(self, comm, function, args=(), kwargs=None, wait=True):
"""Run a function within this process
See Also
--------
Client.run_on_scheduler
"""
from distributed.worker import run
if not dask.config.get("distributed.scheduler.pickle"):
raise ValueError(
"Cannot run function as the scheduler has been explicitly disallowed from "
"deserializing arbitrary bytestrings using pickle via the "
"'distributed.scheduler.pickle' configuration setting."
)
kwargs = kwargs or {}
self.log_event("all", {"action": "run-function", "function": function})
return run(self, comm, function=function, args=args, kwargs=kwargs, wait=wait)
def set_metadata(self, keys=None, value=None):
metadata = self.task_metadata
for key in keys[:-1]:
if key not in metadata or not isinstance(metadata[key], (dict, list)):
metadata[key] = {}
metadata = metadata[key]
metadata[keys[-1]] = value
def get_metadata(self, keys, default=no_default):
metadata = self.task_metadata
for key in keys[:-1]:
metadata = metadata[key]
try:
return metadata[keys[-1]]
except KeyError:
if default != no_default:
return default
else:
raise
def set_restrictions(self, worker: dict[str, Collection[str] | str]):
for key, restrictions in worker.items():
ts = self.tasks[key]
if isinstance(restrictions, str):
restrictions = {restrictions}
ts.worker_restrictions = set(restrictions)
@log_errors
def get_task_prefix_states(self):
state = {}
for tp in self.task_prefixes.values():
active_states = tp.active_states
if any(
active_states.get(s)
for s in {"memory", "erred", "released", "processing", "waiting"}
):
state[tp.name] = {
"memory": active_states["memory"],
"erred": active_states["erred"],
"released": active_states["released"],
"processing": active_states["processing"],
"waiting": active_states["waiting"],
}
return state
def get_task_status(self, keys=None):
return {
key: (self.tasks[key].state if key in self.tasks else None) for key in keys
}
def get_task_stream(self, start=None, stop=None, count=None):
from distributed.diagnostics.task_stream import TaskStreamPlugin
if TaskStreamPlugin.name not in self.plugins:
self.add_plugin(TaskStreamPlugin(self))
plugin = self.plugins[TaskStreamPlugin.name]
return plugin.collect(start=start, stop=stop, count=count)
def start_task_metadata(self, name=None):
plugin = CollectTaskMetaDataPlugin(scheduler=self, name=name)
self.add_plugin(plugin)
def stop_task_metadata(self, name=None):
plugins = [
p
for p in list(self.plugins.values())
if isinstance(p, CollectTaskMetaDataPlugin) and p.name == name
]
if len(plugins) != 1:
raise ValueError(
"Expected to find exactly one CollectTaskMetaDataPlugin "
f"with name {name} but found {len(plugins)}."
)
plugin = plugins[0]
self.remove_plugin(name=plugin.name)
return {"metadata": plugin.metadata, "state": plugin.state}
[docs] async def register_worker_plugin(self, comm, plugin, name=None):
"""Registers a worker plugin on all running and future workers"""
self.worker_plugins[name] = plugin
responses = await self.broadcast(
msg=dict(op="plugin-add", plugin=plugin, name=name)
)
return responses
[docs] async def unregister_worker_plugin(self, comm, name):
"""Unregisters a worker plugin"""
try:
self.worker_plugins.pop(name)
except KeyError:
raise ValueError(f"The worker plugin {name} does not exists")
responses = await self.broadcast(msg=dict(op="plugin-remove", name=name))
return responses
[docs] async def register_nanny_plugin(self, comm, plugin, name=None):
"""Registers a setup function, and call it on every worker"""
self.nanny_plugins[name] = plugin
responses = await self.broadcast(
msg=dict(op="plugin_add", plugin=plugin, name=name),
nanny=True,
)
return responses
[docs] async def unregister_nanny_plugin(self, comm, name):
"""Unregisters a worker plugin"""
try:
self.nanny_plugins.pop(name)
except KeyError:
raise ValueError(f"The nanny plugin {name} does not exists")
responses = await self.broadcast(
msg=dict(op="plugin_remove", name=name), nanny=True
)
return responses
[docs] def transition(self, key, finish: str, *args, stimulus_id: str, **kwargs):
"""Transition a key from its current state to the finish state
Examples
--------
>>> self.transition('x', 'waiting')
{'x': 'processing'}
Returns
-------
Dictionary of recommendations for future transitions
See Also
--------
Scheduler.transitions: transitive version of this function
"""
a: tuple = self._transition(key, finish, stimulus_id, *args, **kwargs)
recommendations, client_msgs, worker_msgs = a
self.send_all(client_msgs, worker_msgs)
return recommendations
[docs] def transitions(self, recommendations: dict, stimulus_id: str):
"""Process transitions until none are left
This includes feedback from previous transitions and continues until we
reach a steady state
"""
client_msgs: dict = {}
worker_msgs: dict = {}
self._transitions(recommendations, client_msgs, worker_msgs, stimulus_id)
self.send_all(client_msgs, worker_msgs)
[docs] def story(self, *keys_or_tasks_or_stimuli: str | TaskState) -> list[tuple]:
"""Get all transitions that touch one of the input keys or stimulus_id's"""
keys_or_stimuli = {
key.key if isinstance(key, TaskState) else key
for key in keys_or_tasks_or_stimuli
}
return scheduler_story(keys_or_stimuli, self.transition_log)
async def get_story(self, keys_or_stimuli: Iterable[str]) -> list[tuple]:
return self.story(*keys_or_stimuli)
transition_story = story
[docs] def reschedule(self, key=None, worker=None):
"""Reschedule a task
Things may have shifted and this task may now be better suited to run
elsewhere
"""
try:
ts = self.tasks[key]
except KeyError:
logger.warning(
"Attempting to reschedule task {}, which was not "
"found on the scheduler. Aborting reschedule.".format(key)
)
return
if ts.state != "processing":
return
if worker and ts.processing_on.address != worker:
return
self.transitions({key: "released"}, f"reschedule-{time()}")
#####################
# Utility functions #
#####################
def add_resources(self, worker: str, resources=None):
ws: WorkerState = self.workers[worker]
if resources:
ws.resources.update(resources)
ws.used_resources = {}
for resource, quantity in ws.resources.items():
ws.used_resources[resource] = 0
dr = self.resources.get(resource, None)
if dr is None:
self.resources[resource] = dr = {}
dr[worker] = quantity
return "OK"
def remove_resources(self, worker):
ws: WorkerState = self.workers[worker]
for resource, quantity in ws.resources.items():
dr: dict = self.resources.get(resource, None)
if dr is None:
self.resources[resource] = dr = {}
del dr[worker]
[docs] def coerce_address(self, addr, resolve=True):
"""
Coerce possible input addresses to canonical form.
*resolve* can be disabled for testing with fake hostnames.
Handles strings, tuples, or aliases.
"""
# XXX how many address-parsing routines do we have?
if addr in self.aliases:
addr = self.aliases[addr]
if isinstance(addr, tuple):
addr = unparse_host_port(*addr)
if not isinstance(addr, str):
raise TypeError(f"addresses should be strings or tuples, got {addr!r}")
if resolve:
addr = resolve_address(addr)
else:
addr = normalize_address(addr)
return addr
[docs] def workers_list(self, workers):
"""
List of qualifying workers
Takes a list of worker addresses or hostnames.
Returns a list of all worker addresses that match
"""
if workers is None:
return list(self.workers)
out = set()
for w in workers:
if ":" in w:
out.add(w)
else:
out.update({ww for ww in self.workers if w in ww}) # TODO: quadratic
return list(out)
async def get_profile(
self,
comm=None,
workers=None,
scheduler=False,
server=False,
merge_workers=True,
start=None,
stop=None,
key=None,
):
if workers is None:
workers = self.workers
else:
workers = set(self.workers) & set(workers)
if scheduler:
return profile.get_profile(self.io_loop.profile, start=start, stop=stop)
results = await asyncio.gather(
*(
self.rpc(w).profile(start=start, stop=stop, key=key, server=server)
for w in workers
),
return_exceptions=True,
)
results = [r for r in results if not isinstance(r, Exception)]
if merge_workers:
response = profile.merge(*results)
else:
response = dict(zip(workers, results))
return response
async def get_profile_metadata(
self,
workers: "Iterable[str] | None" = None,
start: float = 0,
stop: "float | None" = None,
profile_cycle_interval: "str | float | None" = None,
):
dt = profile_cycle_interval or dask.config.get(
"distributed.worker.profile.cycle"
)
dt = parse_timedelta(dt, default="ms")
if workers is None:
workers = self.workers
else:
workers = set(self.workers) & set(workers)
results = await asyncio.gather(
*(self.rpc(w).profile_metadata(start=start, stop=stop) for w in workers),
return_exceptions=True,
)
results = [r for r in results if not isinstance(r, Exception)]
counts = [
(time, sum(pluck(1, group)))
for time, group in itertools.groupby(
merge_sorted(
*(v["counts"] for v in results),
),
lambda t: t[0] // dt * dt,
)
]
keys: dict[str, list[list]] = {
k: [] for v in results for t, d in v["keys"] for k in d
}
groups1 = [v["keys"] for v in results]
groups2 = list(merge_sorted(*groups1, key=first))
last = 0
for t, d in groups2:
tt = t // dt * dt
if tt > last:
last = tt
for k, v in keys.items():
v.append([tt, 0])
for k, v in d.items():
keys[k][-1][1] += v
return {"counts": counts, "keys": keys}
async def performance_report(
self, start: float, last_count: int, code="", mode=None
):
stop = time()
# Profiles
compute, scheduler, workers = await asyncio.gather(
*[
self.get_profile(start=start),
self.get_profile(scheduler=True, start=start),
self.get_profile(server=True, start=start),
]
)
from distributed import profile
def profile_to_figure(state):
data = profile.plot_data(state)
figure, source = profile.plot_figure(data, sizing_mode="stretch_both")
return figure
compute, scheduler, workers = map(
profile_to_figure, (compute, scheduler, workers)
)
# Task stream
task_stream = self.get_task_stream(start=start)
total_tasks = len(task_stream)
timespent: defaultdict[str, float] = defaultdict(float)
for d in task_stream:
for x in d["startstops"]:
timespent[x["action"]] += x["stop"] - x["start"]
tasks_timings = ""
for k in sorted(timespent.keys()):
tasks_timings += f"\n<li> {k} time: {format_time(timespent[k])} </li>"
from distributed.dashboard.components.scheduler import task_stream_figure
from distributed.diagnostics.task_stream import rectangles
rects = rectangles(task_stream)
source, task_stream = task_stream_figure(sizing_mode="stretch_both")
source.data.update(rects)
# Bandwidth
from distributed.dashboard.components.scheduler import (
BandwidthTypes,
BandwidthWorkers,
)
bandwidth_workers = BandwidthWorkers(self, sizing_mode="stretch_both")
bandwidth_workers.update()
bandwidth_types = BandwidthTypes(self, sizing_mode="stretch_both")
bandwidth_types.update()
# System monitor
from distributed.dashboard.components.shared import SystemMonitor
sysmon = SystemMonitor(self, last_count=last_count, sizing_mode="stretch_both")
sysmon.update()
# Scheduler logs
from distributed.dashboard.components.scheduler import SchedulerLogs
logs = SchedulerLogs(self, start=start)
from bokeh.models import Div, Panel, Tabs
import distributed
# HTML
ws: WorkerState
html = """
<h1> Dask Performance Report </h1>
<i> Select different tabs on the top for additional information </i>
<h2> Duration: {time} </h2>
<h2> Tasks Information </h2>
<ul>
<li> number of tasks: {ntasks} </li>
{tasks_timings}
</ul>
<h2> Scheduler Information </h2>
<ul>
<li> Address: {address} </li>
<li> Workers: {nworkers} </li>
<li> Threads: {threads} </li>
<li> Memory: {memory} </li>
<li> Dask Version: {dask_version} </li>
<li> Dask.Distributed Version: {distributed_version} </li>
</ul>
<h2> Calling Code </h2>
<pre>
{code}
</pre>
""".format(
time=format_time(stop - start),
ntasks=total_tasks,
tasks_timings=tasks_timings,
address=self.address,
nworkers=len(self.workers),
threads=sum(ws.nthreads for ws in self.workers.values()),
memory=format_bytes(sum(ws.memory_limit for ws in self.workers.values())),
code=code,
dask_version=dask.__version__,
distributed_version=distributed.__version__,
)
html = Div(
text=html,
style={
"width": "100%",
"height": "100%",
"max-width": "1920px",
"max-height": "1080px",
"padding": "12px",
"border": "1px solid lightgray",
"box-shadow": "inset 1px 0 8px 0 lightgray",
"overflow": "auto",
},
)
html = Panel(child=html, title="Summary")
compute = Panel(child=compute, title="Worker Profile (compute)")
workers = Panel(child=workers, title="Worker Profile (administrative)")
scheduler = Panel(child=scheduler, title="Scheduler Profile (administrative)")
task_stream = Panel(child=task_stream, title="Task Stream")
bandwidth_workers = Panel(
child=bandwidth_workers.root, title="Bandwidth (Workers)"
)
bandwidth_types = Panel(child=bandwidth_types.root, title="Bandwidth (Types)")
system = Panel(child=sysmon.root, title="System")
logs = Panel(child=logs.root, title="Scheduler Logs")
tabs = Tabs(
tabs=[
html,
task_stream,
system,
logs,
compute,
workers,
scheduler,
bandwidth_workers,
bandwidth_types,
]
)
from bokeh.core.templates import get_env
from bokeh.plotting import output_file, save
with tmpfile(extension=".html") as fn:
output_file(filename=fn, title="Dask Performance Report", mode=mode)
template_directory = os.path.join(
os.path.dirname(os.path.abspath(__file__)), "dashboard", "templates"
)
template_environment = get_env()
template_environment.loader.searchpath.append(template_directory)
template = template_environment.get_template("performance_report.html")
save(tabs, filename=fn, template=template)
with open(fn) as f:
data = f.read()
return data
async def get_worker_logs(self, n=None, workers=None, nanny=False):
results = await self.broadcast(
msg={"op": "get_logs", "n": n}, workers=workers, nanny=nanny
)
return results
def log_event(self, topic: str | Collection[str], msg: Any) -> None:
event = (time(), msg)
if not isinstance(topic, str):
for t in topic:
self.events[t].append(event)
self.event_counts[t] += 1
self._report_event(t, event)
else:
self.events[topic].append(event)
self.event_counts[topic] += 1
self._report_event(topic, event)
for plugin in list(self.plugins.values()):
try:
plugin.log_event(topic, msg)
except Exception:
logger.info("Plugin failed with exception", exc_info=True)
def _report_event(self, name, event):
for client in self.event_subscriber[name]:
self.report(
{
"op": "event",
"topic": name,
"event": event,
},
client=client,
)
def subscribe_topic(self, topic, client):
self.event_subscriber[topic].add(client)
def unsubscribe_topic(self, topic, client):
self.event_subscriber[topic].discard(client)
def get_events(self, topic=None):
if topic is not None:
return tuple(self.events[topic])
else:
return valmap(tuple, self.events)
async def get_worker_monitor_info(self, recent=False, starts=None):
if starts is None:
starts = {}
results = await asyncio.gather(
*(
self.rpc(w).get_monitor_info(recent=recent, start=starts.get(w, 0))
for w in self.workers
)
)
return dict(zip(self.workers, results))
###########
# Cleanup #
###########
[docs] async def reevaluate_occupancy(self, worker_index: int = 0):
"""Periodically reassess task duration time
The expected duration of a task can change over time. Unfortunately we
don't have a good constant-time way to propagate the effects of these
changes out to the summaries that they affect, like the total expected
runtime of each of the workers, or what tasks are stealable.
In this coroutine we walk through all of the workers and re-align their
estimates with the current state of tasks. We do this periodically
rather than at every transition, and we only do it if the scheduler
process isn't under load (using psutil.Process.cpu_percent()). This
lets us avoid this fringe optimization when we have better things to
think about.
"""
try:
while self.status != Status.closed:
last = time()
delay = 0.1
if self.proc.cpu_percent() < 50:
workers: list = list(self.workers.values())
nworkers: int = len(workers)
i: int
for i in range(nworkers):
ws: WorkerState = workers[worker_index % nworkers]
worker_index += 1
try:
if ws is None or not ws.processing:
continue
self._reevaluate_occupancy_worker(ws)
finally:
del ws # lose ref
duration = time() - last
if duration > 0.005: # 5ms since last release
delay = duration * 5 # 25ms gap
break
await asyncio.sleep(delay)
except Exception:
logger.error("Error in reevaluate occupancy", exc_info=True)
raise
async def check_worker_ttl(self):
now = time()
stimulus_id = f"check-worker-ttl-{now}"
for ws in self.workers.values():
if (ws.last_seen < now - self.worker_ttl) and (
ws.last_seen < now - 10 * heartbeat_interval(len(self.workers))
):
logger.warning(
"Worker failed to heartbeat within %s seconds. Closing: %s",
self.worker_ttl,
ws,
)
await self.remove_worker(address=ws.address, stimulus_id=stimulus_id)
def check_idle(self):
assert self.idle_timeout
if self.status in (Status.closing, Status.closed):
return
if self.transition_counter != self._idle_transition_counter:
self._idle_transition_counter = self.transition_counter
self.idle_since = None
return
if any([ws.processing for ws in self.workers.values()]) or self.unrunnable:
self.idle_since = None
return
if not self.idle_since:
self.idle_since = time()
if time() > self.idle_since + self.idle_timeout:
assert self.idle_since
logger.info(
"Scheduler closing after being idle for %s",
format_time(self.idle_timeout),
)
self._ongoing_background_tasks.call_soon(self.close)
[docs] def adaptive_target(self, target_duration=None):
"""Desired number of workers based on the current workload
This looks at the current running tasks and memory use, and returns a
number of desired workers. This is often used by adaptive scheduling.
Parameters
----------
target_duration : str
A desired duration of time for computations to take. This affects
how rapidly the scheduler will ask to scale.
See Also
--------
distributed.deploy.Adaptive
"""
if target_duration is None:
target_duration = dask.config.get("distributed.adaptive.target-duration")
target_duration = parse_timedelta(target_duration)
# CPU
cpu = math.ceil(
self.total_occupancy / target_duration
) # TODO: threads per worker
# Avoid a few long tasks from asking for many cores
tasks_processing = 0
for ws in self.workers.values():
tasks_processing += len(ws.processing)
if tasks_processing > cpu:
break
else:
cpu = min(tasks_processing, cpu)
if self.unrunnable and not self.workers:
cpu = max(1, cpu)
# add more workers if more than 60% of memory is used
limit = sum(ws.memory_limit for ws in self.workers.values())
used = sum(ws.nbytes for ws in self.workers.values())
memory = 0
if used > 0.6 * limit and limit > 0:
memory = 2 * len(self.workers)
target = max(memory, cpu)
if target >= len(self.workers):
return target
else: # Scale down?
to_close = self.workers_to_close()
return len(self.workers) - len(to_close)
[docs] def request_acquire_replicas(
self, addr: str, keys: Iterable[str], *, stimulus_id: str
) -> None:
"""Asynchronously ask a worker to acquire a replica of the listed keys from
other workers. This is a fire-and-forget operation which offers no feedback for
success or failure, and is intended for housekeeping and not for computation.
"""
who_has = {}
nbytes = {}
for key in keys:
ts = self.tasks[key]
assert ts.who_has
who_has[key] = [ws.address for ws in ts.who_has]
nbytes[key] = ts.nbytes
self.stream_comms[addr].send(
{
"op": "acquire-replicas",
"who_has": who_has,
"nbytes": nbytes,
"stimulus_id": stimulus_id,
},
)
[docs] def request_remove_replicas(
self, addr: str, keys: list[str], *, stimulus_id: str
) -> None:
"""Asynchronously ask a worker to discard its replica of the listed keys.
This must never be used to destroy the last replica of a key. This is a
fire-and-forget operation, intended for housekeeping and not for computation.
The replica disappears immediately from TaskState.who_has on the Scheduler side;
if the worker refuses to delete, e.g. because the task is a dependency of
another task running on it, it will (also asynchronously) inform the scheduler
to re-add itself to who_has. If the worker agrees to discard the task, there is
no feedback.
"""
ws = self.workers[addr]
# The scheduler immediately forgets about the replica and suggests the worker to
# drop it. The worker may refuse, at which point it will send back an add-keys
# message to reinstate it.
for key in keys:
ts = self.tasks[key]
if self.validate:
# Do not destroy the last copy
assert len(ts.who_has) > 1
self.remove_replica(ts, ws)
self.stream_comms[addr].send(
{
"op": "remove-replicas",
"keys": keys,
"stimulus_id": stimulus_id,
}
)
def _remove_from_processing(state: SchedulerState, ts: TaskState) -> str | None:
"""Remove *ts* from the set of processing tasks.
See also
--------
Scheduler._set_duration_estimate
"""
ws = ts.processing_on
assert ws
ts.processing_on = None
if ws.address not in state.workers: # may have been removed
return None
duration = ws.processing.pop(ts)
ws.long_running.discard(ts)
if not ws.processing:
state.total_occupancy -= ws.occupancy
ws.occupancy = 0
else:
state.total_occupancy -= duration
ws.occupancy -= duration
state.check_idle_saturated(ws)
state.release_resources(ts, ws)
return ws.address
def _add_to_memory(
state: SchedulerState,
ts: TaskState,
ws: WorkerState,
recommendations: dict[str, str],
client_msgs: dict[str, list[dict[str, str]]],
type=None,
typename: str | None = None,
) -> None:
"""Add ts to the set of in-memory tasks"""
if state.validate:
assert ts not in ws.has_what
state.add_replica(ts, ws)
deps = list(ts.dependents)
if len(deps) > 1:
deps.sort(key=operator.attrgetter("priority"), reverse=True)
for dts in deps:
s = dts.waiting_on
if ts in s:
s.discard(ts)
if not s: # new task ready to run
recommendations[dts.key] = "processing"
for dts in ts.dependencies:
s = dts.waiters
s.discard(ts)
if not s and not dts.who_wants:
recommendations[dts.key] = "released"
report_msg = {}
if not ts.waiters and not ts.who_wants:
recommendations[ts.key] = "released"
else:
report_msg["op"] = "key-in-memory"
report_msg["key"] = ts.key
if type is not None:
report_msg["type"] = type
for cs in ts.who_wants:
client_msgs[cs.client_key] = [report_msg]
ts.state = "memory"
ts.type = typename # type: ignore
ts.group.types.add(typename) # type: ignore
cs = state.clients["fire-and-forget"]
if ts in cs.wants_what:
_client_releases_keys(
state,
cs=cs,
keys=[ts.key],
recommendations=recommendations,
)
def _propagate_forgotten(
state: SchedulerState,
ts: TaskState,
recommendations: dict[str, str],
worker_msgs: dict[str, list[dict[str, Any]]],
stimulus_id: str,
) -> None:
ts.state = "forgotten"
for dts in ts.dependents:
dts.has_lost_dependencies = True
dts.dependencies.remove(ts)
dts.waiting_on.discard(ts)
if dts.state not in ("memory", "erred"):
# Cannot compute task anymore
recommendations[dts.key] = "forgotten"
ts.dependents.clear()
ts.waiters.clear()
for dts in ts.dependencies:
dts.dependents.remove(ts)
dts.waiters.discard(ts)
if not dts.dependents and not dts.who_wants:
# Task not needed anymore
assert dts is not ts
recommendations[dts.key] = "forgotten"
ts.dependencies.clear()
ts.waiting_on.clear()
for ws in ts.who_has:
if ws.address in state.workers: # in case worker has died
worker_msgs[ws.address] = [
{
"op": "free-keys",
"keys": [ts.key],
"stimulus_id": stimulus_id,
}
]
state.remove_all_replicas(ts)
def _client_releases_keys(
state: SchedulerState,
keys: Collection[str],
cs: ClientState,
recommendations: dict[str, str],
) -> None:
"""Remove keys from client desired list"""
logger.debug("Client %s releases keys: %s", cs.client_key, keys)
for key in keys:
ts = state.tasks.get(key)
if ts is not None and ts in cs.wants_what:
cs.wants_what.remove(ts)
ts.who_wants.remove(cs)
if not ts.who_wants:
if not ts.dependents:
# No live dependents, can forget
recommendations[ts.key] = "forgotten"
elif ts.state != "erred" and not ts.waiters:
recommendations[ts.key] = "released"
def _task_to_msg(
state: SchedulerState, ts: TaskState, duration: float = -1
) -> dict[str, Any]:
"""Convert a single computational task to a message"""
# FIXME: The duration attribute is not used on worker. We could save ourselves the