From afedccd1faa38780d8dcde2d6b7b245f70bffdb9 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 15 Jun 2022 18:27:03 -0600 Subject: [PATCH 01/98] unused: `OrderedSet` collection Idea was that if a `SortedSet` of unrunnable tasks is too expensive, then insertion order is probably _approximately_ priority order, since higher-priority (root) tasks will be scheduled first. This would give us O(1) for all necessary operations, instead of O(logn) for adding and removing. Interestingly, the SortedSet implementation could be hacked to support O(1) `pop` and `popleft`, and inserting a min/max value. In the most common case (root tasks), we're always inserting a value that's greater than the max. Something like this might be the best tradeoff, since it gives us O(1) in the common case but still maintains the sorted gaurantee, which is easier to reason about. --- distributed/collections.py | 116 ++++++++++++++++++++++++++++++++++++- 1 file changed, 114 insertions(+), 2 deletions(-) diff --git a/distributed/collections.py b/distributed/collections.py index c25001047f6..202f3580bdb 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -4,8 +4,14 @@ import weakref from collections import OrderedDict, UserDict from collections.abc import Callable, Hashable, Iterator -from typing import MutableSet # TODO move to collections.abc (requires Python >=3.9) -from typing import Any, TypeVar, cast +from typing import ( # TODO move to collections.abc (requires Python >=3.9) + AbstractSet, + Any, + Iterable, + MutableSet, + TypeVar, + cast, +) T = TypeVar("T", bound=Hashable) @@ -112,3 +118,109 @@ def sorted(self) -> Iterator[T]: def clear(self) -> None: self._data.clear() self._heap.clear() + + +class OrderedSet(MutableSet[T]): + """ + A insertion-ordered set. + + All operations are O(1) complexity. + + Equality tests between OrderedSet objects are order-sensitive. Equality tests + between OrderedSet objects and other AbstractSet objects are order-insensitive like + regular sets. + """ + + __slots__ = ("_data",) + _data: dict[T, None] + + def __init__(self, iterable: Iterable[T] | None = None) -> None: + if iterable: + self._data = dict.fromkeys(iterable) + else: + self._data = {} + + def add(self, value: T) -> None: + self._data[value] = None + # NOTE: updating an already-existing item in a dict does not change iteration order + + def discard(self, value: T) -> None: + self._data.pop(value, None) + + def clear(self) -> None: + self._data.clear() + + def copy(self) -> OrderedSet[T]: + new = type(self)() + new._data = self._data.copy() + return new + + def pop(self) -> T: + "Remove and return the last-inserted item" + if not self._data: + raise KeyError("pop on an empty set") + return self._data.popitem()[0] + + def popleft(self) -> T: + "Remove and return the first-inserted item" + if not self._data: + raise KeyError("popleft on an empty set") + first = next(iter(self._data)) + self._data.pop(first) + return first + + def peek(self) -> T: + if not self._data: + raise KeyError("peek into empty set") + return next(reversed(self._data)) + + def peekleft(self) -> T: + if not self._data: + raise KeyError("peekleft into empty set") + return next(iter(self._data)) + + def rotate(self, n=1) -> None: + """ + Rotate the OrderedSet ``n`` steps to the right. + + Note that each rotation is an O(1) operation, so the time-complexity + is equivalent to ``n``. + """ + if n == 0: + return + if n < 0: + raise ValueError(f"{type(self).__name__} can only be rotated to the right") + n = n % len(self) + for _ in range(n): + self.add(self.popleft()) + + def update(self, iterable: Iterable[T]) -> None: + for x in iterable: + self._data[x] = None + + def __repr__(self) -> str: + return f"{type(self).__name__}({', '.join(map(str, self))})>" + + def __contains__(self, value: object) -> bool: + return value in self._data + + def __len__(self) -> int: + return len(self._data) + + def __iter__(self) -> Iterator[T]: + """Iterate over all elements in insertion order.""" + return iter(self._data) + + def __reverse__(self) -> Iterator[T]: + """Iterate over all elements in reverse insertion order.""" + return reversed(self._data) + + def __eq__(self, other: object) -> bool: + if isinstance(other, type(self)): + return len(other._data) == len(self._data) and all( + a == b for a, b in zip(self._data, other._data) + ) + if isinstance(other, AbstractSet): + return self._data.keys() == other + + return NotImplemented From 6b6651b95e18b11c5e997fee84e2bd02e757eeee Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 13:20:04 -0600 Subject: [PATCH 02/98] Queue root tasks scheduler-side --- distributed/collections.py | 3 + distributed/dashboard/components/scheduler.py | 9 +- distributed/scheduler.py | 235 +++++++++++++++++- distributed/tests/test_scheduler.py | 43 +++- .../widgets/templates/worker_state.html.j2 | 1 + 5 files changed, 280 insertions(+), 11 deletions(-) diff --git a/distributed/collections.py b/distributed/collections.py index 202f3580bdb..4c978ebf0d7 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -62,6 +62,9 @@ def __repr__(self) -> str: def __contains__(self, value: object) -> bool: return value in self._data + def __bool__(self) -> bool: + return bool(self._data) + def __len__(self) -> int: return len(self._data) diff --git a/distributed/dashboard/components/scheduler.py b/distributed/dashboard/components/scheduler.py index 5617282f089..f5685b7a277 100644 --- a/distributed/dashboard/components/scheduler.py +++ b/distributed/dashboard/components/scheduler.py @@ -3105,6 +3105,10 @@ def __init__(self, scheduler, **kwargs): Ready:  @processing +
+ Queued:  + @queued +
""", ) self.root.add_tools(hover) @@ -3118,6 +3122,7 @@ def update(self): "released": {}, "processing": {}, "waiting": {}, + "queued": {}, } for tp in self.scheduler.task_prefixes.values(): @@ -3128,6 +3133,7 @@ def update(self): state["released"][tp.name] = active_states["released"] state["processing"][tp.name] = active_states["processing"] state["waiting"][tp.name] = active_states["waiting"] + state["queued"][tp.name] = active_states["queued"] state["all"] = {k: sum(v[k] for v in state.values()) for k in state["memory"]} @@ -3140,7 +3146,7 @@ def update(self): totals = { k: sum(state[k].values()) - for k in ["all", "memory", "erred", "released", "waiting"] + for k in ["all", "memory", "erred", "released", "waiting", "queued"] } totals["processing"] = totals["all"] - sum( v for k, v in totals.items() if k != "all" @@ -3149,6 +3155,7 @@ def update(self): self.root.title.text = ( "Progress -- total: %(all)s, " "in-memory: %(memory)s, processing: %(processing)s, " + "queued: %(queued)s, " "waiting: %(waiting)s, " "erred: %(erred)s" % totals ) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 34f30938a51..13ecb35d8f9 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -58,6 +58,7 @@ from distributed._stories import scheduler_story from distributed.active_memory_manager import ActiveMemoryManagerExtension, RetireWorker from distributed.batched import BatchedSend +from distributed.collections import HeapSet from distributed.comm import ( Comm, CommClosedError, @@ -129,7 +130,15 @@ "stealing": WorkStealing, } -ALL_TASK_STATES = {"released", "waiting", "no-worker", "processing", "erred", "memory"} +ALL_TASK_STATES = { + "released", + "waiting", + "no-worker", + "queued", + "processing", + "erred", + "memory", +} class ClientState: @@ -440,6 +449,10 @@ class WorkerState: #: been running. executing: dict[TaskState, float] + #: Tasks queued to _potentially_ run on this worker in the future, ordered by priority. + #: The queuing is scheduler-side only; the worker is unaware of these tasks. + queued: HeapSet[TaskState] + #: 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. @@ -494,6 +507,7 @@ def __init__( self.processing = {} self.long_running = set() self.executing = {} + self.queued = HeapSet(key=operator.attrgetter("priority")) self.resources = {} self.used_resources = {} self.extra = extra or {} @@ -563,7 +577,8 @@ def __repr__(self) -> str: f"" + f"processing: {len(self.processing)}, " + f"queued: {len(self.queued)}>" ) def _repr_html_(self): @@ -573,6 +588,7 @@ def _repr_html_(self): status=self.status.name, has_what=self.has_what, processing=self.processing, + queued=self.queued, ) def identity(self) -> dict[str, Any]: @@ -972,6 +988,10 @@ class TaskState: #: it. This attribute is kept in sync with :attr:`WorkerState.processing`. processing_on: WorkerState | None + #: If this task is in the "queued" state, which worker is currently queued + #: it. This attribute is kept in sync with :attr:`WorkerState.queued`. + queued_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 @@ -1093,6 +1113,7 @@ def __init__(self, key: str, run_spec: object): self.waiters = set() self.who_has = set() self.processing_on = None + self.queued_on = None self.has_lost_dependencies = False self.host_restrictions = None # type: ignore self.worker_restrictions = None # type: ignore @@ -1801,6 +1822,27 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: ws if tg.states["released"] + tg.states["waiting"] > 1 else None ) tg.last_worker_tasks_left -= 1 + + # Queue if worker is full to avoid root task overproduction. + if len(ws.processing) >= ws.nthreads: + # TODO this should be a transition function instead. + # But how do we get the `ws` into it? Recommendations on the scheduler can't take arguments. + + if self.validate: + assert not ts.queued_on, ts.queued_on + assert ts not in ws.queued + + # TODO maintain global queue of tasks as well for newly arriving workers to use? + # QUESTION could `queued` be an OrderedSet instead of a HeapSet, giving us O(1) + # operations instead of O(logn)? Reasoning is that we're always inserting elements + # in priority order anyway. + # This wouldn't work in the case that a batch of lower-priority root tasks becomes + # ready before a batch of higher-priority root tasks. + ws.queued.add(ts) + ts.queued_on = ws + ts.state = "queued" + return None + return ws if ts.dependencies or valid_workers is not None: @@ -1849,6 +1891,7 @@ def transition_waiting_processing(self, key, stimulus_id): assert not ts.who_has assert not ts.exception_blame assert not ts.processing_on + assert not ts.queued_on assert not ts.has_lost_dependencies assert ts not in self.unrunnable assert all(dts.who_has for dts in ts.dependencies) @@ -1900,6 +1943,7 @@ def transition_waiting_memory( if self.validate: assert not ts.processing_on + assert not ts.queued_on assert ts.waiting_on assert ts.state == "waiting" @@ -1916,6 +1960,7 @@ def transition_waiting_memory( if self.validate: assert not ts.processing_on + assert not ts.queued_on assert not ts.waiting_on assert ts.who_has @@ -2007,7 +2052,10 @@ def transition_processing_memory( if nbytes is not None: ts.set_nbytes(nbytes) - _remove_from_processing(self, ts) + # NOTE: recommendations for queued tasks are added first, so they'll be popped last, + # allowing higher-priority downstream tasks to be transitioned first. + # FIXME: this would be incorrect if queued tasks are user-annotated as higher priority. + _remove_from_processing(self, ts, recommendations) _add_to_memory( self, ts, ws, recommendations, client_msgs, type=type, typename=typename @@ -2015,7 +2063,18 @@ def transition_processing_memory( if self.validate: assert not ts.processing_on + assert not ts.queued_on assert not ts.waiting_on + processing_recs = { + k: r for k, r in recommendations.items() if r == "processing" + } + assert list(processing_recs) == ( + sr := sorted( + processing_recs, + key=lambda k: self.tasks[k].priority, + reverse=True, + ) + ), (list(processing_recs), sr) return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2038,6 +2097,7 @@ def transition_memory_released(self, key, stimulus_id, safe: bool = False): if self.validate: assert not ts.waiting_on assert not ts.processing_on + assert not ts.queued_on if safe: assert not ts.waiters @@ -2199,6 +2259,7 @@ def transition_waiting_released(self, key, stimulus_id): if self.validate: assert not ts.who_has assert not ts.processing_on + assert not ts.queued_on dts: TaskState for dts in ts.dependencies: @@ -2240,9 +2301,9 @@ def transition_processing_released(self, key, stimulus_id): assert not ts.waiting_on assert self.tasks[key].state == "processing" - w: str = _remove_from_processing(self, ts) - if w: - worker_msgs[w] = [ + ws = _remove_from_processing(self, ts, recommendations) + if ws: + worker_msgs[ws] = [ { "op": "free-keys", "keys": [key], @@ -2267,6 +2328,7 @@ def transition_processing_released(self, key, stimulus_id): if self.validate: assert not ts.processing_on + assert not ts.queued_on return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2309,7 +2371,7 @@ def transition_processing_erred( ws = ts.processing_on ws.actors.remove(ts) - w = _remove_from_processing(self, ts) + w = _remove_from_processing(self, ts, recommendations) ts.erred_on.add(w or worker) # type: ignore if exception is not None: @@ -2358,6 +2420,7 @@ def transition_processing_erred( if self.validate: assert not ts.processing_on + assert not ts.queued_on return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2398,6 +2461,108 @@ def transition_no_worker_released(self, key, stimulus_id): pdb.set_trace() raise + def transition_queued_released(self, key, stimulus_id): + try: + ts: TaskState = self.tasks[key] + recommendations: dict = {} + client_msgs: dict = {} + worker_msgs: dict = {} + + # TODO allow `remove_worker` to clear `queued_on` and `ws.queued` eagerly; it's more efficient. + ws = ts.queued_on + assert ws + + if self.validate: + assert ts in ws.queued + assert not ts.processing_on + + ws.queued.remove(ts) + ts.queued_on = None + + # TODO copied from `transition_processing_released`; factor out into helper function + ts.state = "released" + + if ts.has_lost_dependencies: + recommendations[key] = "forgotten" + elif ts.waiters or ts.who_wants: + # TODO rescheduling of queued root tasks may be poor. + 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() + + return recommendations, client_msgs, worker_msgs + except Exception as e: + logger.exception(e) + if LOG_PDB: + import pdb + + pdb.set_trace() + raise + + def transition_queued_processing(self, key, stimulus_id): + try: + ts: TaskState = self.tasks[key] + recommendations: dict = {} + client_msgs: dict = {} + worker_msgs: dict = {} + + ws = ts.queued_on + assert ws + # TODO should this be a graceful transition to released? I think `remove_worker` + # makes it such that this should never happen. + assert ( + self.workers[ws.address] is ws + ), f"Task {ts} queued on stale worker {ws}" + + if self.validate: + assert not ts.actor, "Actors can't be queued wat" + assert ts in ws.queued + # Copied from `transition_waiting_processing` + assert not ts.processing_on + assert not ts.waiting_on + assert not ts.who_has + assert not ts.exception_blame + assert not ts.has_lost_dependencies + assert ts not in self.unrunnable + assert all(dts.who_has for dts in ts.dependencies) + + # TODO other validation that this is still an appropriate worker? + + if len(ws.processing) < ws.nthreads: + # If more important tasks already got scheduled, remain queued + + ts.queued_on = None + ws.queued.remove(ts) + # TODO Copied from `transition_waiting_processing`; factor out into helper function + 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[ws.address] = [_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 _remove_key(self, key): ts: TaskState = self.tasks.pop(key) assert ts.state == "forgotten" @@ -2421,6 +2586,7 @@ def transition_memory_forgotten(self, key, stimulus_id): if self.validate: assert ts.state == "memory" assert not ts.processing_on + assert not ts.queued_on assert not ts.waiting_on if not ts.run_spec: # It's ok to forget a pure data task @@ -2463,6 +2629,7 @@ def transition_released_forgotten(self, key, stimulus_id): assert ts.state in ("released", "erred") assert not ts.who_has assert not ts.processing_on + assert not ts.queued_on assert not ts.waiting_on, (ts, ts.waiting_on) if not ts.run_spec: # It's ok to forget a pure data task @@ -2503,6 +2670,8 @@ def transition_released_forgotten(self, key, stimulus_id): ("waiting", "released"): transition_waiting_released, ("waiting", "processing"): transition_waiting_processing, ("waiting", "memory"): transition_waiting_memory, + ("queued", "released"): transition_queued_released, + ("queued", "processing"): transition_queued_processing, ("processing", "released"): transition_processing_released, ("processing", "memory"): transition_processing_memory, ("processing", "erred"): transition_processing_erred, @@ -2774,6 +2943,8 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): """ ts: TaskState tasks = [] + # TODO maintain global queue of tasks and reallocate them here + # FIXME queued tasks will not be assigned to a new worker at all!! for ts in self.unrunnable: valid: set = self.valid_workers(ts) if valid is None or ws in valid: @@ -4261,6 +4432,10 @@ async def remove_worker( else: # pure data recommendations[ts.key] = "forgotten" + for ts in ws.queued.sorted(): + recommendations[ts.key] = "released" + # ws.queued.clear() # TODO more performant + self.transitions(recommendations, stimulus_id=stimulus_id) for plugin in list(self.plugins.values()): @@ -4379,6 +4554,7 @@ def validate_released(self, key): assert not ts.waiting_on assert not ts.who_has assert not ts.processing_on + assert not ts.queued_on assert not any([ts in dts.waiters for dts in ts.dependencies]) assert ts not in self.unrunnable @@ -4387,12 +4563,27 @@ def validate_waiting(self, key): assert ts.waiting_on assert not ts.who_has assert not ts.processing_on + assert not ts.queued_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_queued(self, key): + ts: TaskState = self.tasks[key] + dts: TaskState + assert not ts.waiting_on + ws = ts.queued_on + assert ws + assert self.workers.get(ws.address) is ws, f"{ts} queued on stale worker {ws}" + assert ts in ws.queued + assert not ts.who_has + assert not ts.processing_on + for dts in ts.dependencies: + assert dts.who_has + assert ts in dts.waiters + def validate_processing(self, key): ts: TaskState = self.tasks[key] dts: TaskState @@ -4411,6 +4602,7 @@ def validate_memory(self, key): assert ts.who_has assert bool(ts in self.replicated_tasks) == (len(ts.who_has) > 1) assert not ts.processing_on + assert not ts.queued_on assert not ts.waiting_on assert ts not in self.unrunnable for dts in ts.dependents: @@ -4425,6 +4617,7 @@ def validate_no_worker(self, key): assert not ts.waiting_on assert ts in self.unrunnable assert not ts.processing_on + assert not ts.queued_on assert not ts.who_has for dts in ts.dependencies: assert dts.who_has @@ -7155,7 +7348,9 @@ def request_remove_replicas( ) -def _remove_from_processing(state: SchedulerState, ts: TaskState) -> str | None: +def _remove_from_processing( + state: SchedulerState, ts: TaskState, recommendations: dict +) -> str | None: """Remove *ts* from the set of processing tasks. See also @@ -7181,6 +7376,19 @@ def _remove_from_processing(state: SchedulerState, ts: TaskState) -> str | None: state.check_idle_saturated(ws) state.release_resources(ts, ws) + # If a slot has opened up for a queued task, schedule it. + if ws.queued and len(ws.processing) < ws.nthreads: + # TODO peek or pop? + # What if multiple tasks complete on a worker in one transition cycle? Is that possible? + # TODO should we only be scheduling 1 taks? Or N open threads? Is there a possible deadlock + # where tasks remain queued on a worker forever? + qts = ws.queued.peek() + if state.validate: + assert qts.state == "queued" + assert qts.queued_on is ws, (qts.queued_on, ws) + assert qts.key not in recommendations, recommendations[qts.key] + recommendations[qts.key] = "processing" + return ws.address @@ -7458,8 +7666,18 @@ def validate_task_state(ts: TaskState) -> None: assert dts.state != "forgotten" assert (ts.processing_on is not None) == (ts.state == "processing") + assert not (ts.processing_on and ts.queued_on), (ts.processing_on, ts.queued_on) assert bool(ts.who_has) == (ts.state == "memory"), (ts, ts.who_has, ts.state) + if ts.queued_on: + assert ts.state == "queued" + assert ts in ts.queued_on.queued + + if ts.state == "queued": + assert ts.queued_on + assert not ts.processing_on + assert not ts.who_has + if ts.state == "processing": assert all(dts.who_has for dts in ts.dependencies), ( "task processing without all deps", @@ -7467,6 +7685,7 @@ def validate_task_state(ts: TaskState) -> None: str(ts.dependencies), ) assert not ts.waiting_on + assert not ts.queued_on if ts.who_has: assert ts.waiters or ts.who_wants, ( diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 4c3d43ab711..125f09a8827 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -16,12 +16,12 @@ import cloudpickle import psutil import pytest -from tlz import concat, first, merge, valmap +from tlz import concat, first, merge, partition, valmap from tornado.ioloop import IOLoop, PeriodicCallback import dask from dask import delayed -from dask.utils import apply, parse_timedelta, stringify, tmpfile, typename +from dask.utils import apply, parse_bytes, parse_timedelta, stringify, tmpfile, typename from distributed import ( CancelledError, @@ -54,6 +54,7 @@ raises_with_cause, slowadd, slowdec, + slowidentity, slowinc, tls_only_security, varying, @@ -245,6 +246,44 @@ def random(**kwargs): test_decide_worker_coschedule_order_neighbors_() +@pytest.mark.slow +@gen_cluster( + client=True, + nthreads=[("127.0.0.1", 2)] * 2, + worker_kwargs={"memory_limit": "1.0GiB"}, + timeout=3600, # TODO remove + Worker=Nanny, + scheduler_kwargs=dict( # TODO remove + dashboard=True, + dashboard_address=":8787", + ), + config={ + "distributed.worker.memory.target": False, + "distributed.worker.memory.spill": False, + "distributed.scheduler.work-stealing": False, + }, +) +async def test_root_task_overproduction(c, s, *nannies): + """ + Workload that would run out of memory and kill workers if >2 root tasks were + ever in memory at once on a worker. + """ + + @delayed(pure=True) + def big_data(size: int) -> str: + return "x" * size + + roots = [ + big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(16) + ] + passthrough = [delayed(slowidentity)(x) for x in roots] + memory_consumed = [delayed(len)(x) for x in passthrough] + reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] + final = sum(reduction) + + await c.compute(final) + + @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) async def test_move_data_over_break_restrictions(client, s, a, b, c): [x] = await client.scatter([1], workers=b.address) diff --git a/distributed/widgets/templates/worker_state.html.j2 b/distributed/widgets/templates/worker_state.html.j2 index cd152080bfc..08629998e1f 100644 --- a/distributed/widgets/templates/worker_state.html.j2 +++ b/distributed/widgets/templates/worker_state.html.j2 @@ -3,3 +3,4 @@ status: {{ status }} memory: {{ has_what | length }} processing: {{ processing | length }} + queued: {{ queued | length }} From 6225d1a56cc6123f98636d313d45091189d0b600 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 13:37:54 -0600 Subject: [PATCH 03/98] Show queued tasks with crosshatching on dashboard --- distributed/dashboard/components/scheduler.py | 14 +++++- distributed/diagnostics/progress_stream.py | 49 ++++++++++++------- 2 files changed, 45 insertions(+), 18 deletions(-) diff --git a/distributed/dashboard/components/scheduler.py b/distributed/dashboard/components/scheduler.py index f5685b7a277..8aee47b26c9 100644 --- a/distributed/dashboard/components/scheduler.py +++ b/distributed/dashboard/components/scheduler.py @@ -2986,7 +2986,7 @@ def __init__(self, scheduler, **kwargs): self.scheduler = scheduler data = progress_quads( - dict(all={}, memory={}, erred={}, released={}, processing={}) + dict(all={}, memory={}, erred={}, released={}, processing={}, queued={}) ) self.source = ColumnDataSource(data=data) @@ -3058,6 +3058,18 @@ def __init__(self, scheduler, **kwargs): fill_alpha=0.35, line_alpha=0, ) + self.root.quad( + source=self.source, + top="top", + bottom="bottom", + left="processing-loc", + right="queued-loc", + fill_color="gray", + hatch_pattern="/", + hatch_color="white", + fill_alpha=0.35, + line_alpha=0, + ) self.root.text( source=self.source, text="show-name", diff --git a/distributed/diagnostics/progress_stream.py b/distributed/diagnostics/progress_stream.py index 45f84b1c2cd..04aa7c7c596 100644 --- a/distributed/diagnostics/progress_stream.py +++ b/distributed/diagnostics/progress_stream.py @@ -64,23 +64,29 @@ def progress_quads(msg, nrows=8, ncols=3): ... 'memory': {'inc': 2, 'dec': 0, 'add': 1}, ... 'erred': {'inc': 0, 'dec': 1, 'add': 0}, ... 'released': {'inc': 1, 'dec': 0, 'add': 1}, - ... 'processing': {'inc': 1, 'dec': 0, 'add': 2}} + ... 'processing': {'inc': 1, 'dec': 0, 'add': 2}, + ... 'queued': {'inc': 1, 'dec': 0, 'add': 2}} >>> progress_quads(msg, nrows=2) # doctest: +SKIP - {'name': ['inc', 'add', 'dec'], - 'left': [0, 0, 1], - 'right': [0.9, 0.9, 1.9], - 'top': [0, -1, 0], - 'bottom': [-.8, -1.8, -.8], - 'released': [1, 1, 0], - 'memory': [2, 1, 0], - 'erred': [0, 0, 1], - 'processing': [1, 0, 2], - 'done': ['3 / 5', '2 / 4', '1 / 1'], - 'released-loc': [.2/.9, .25 / 0.9, 1], - 'memory-loc': [3 / 5 / .9, .5 / 0.9, 1], - 'erred-loc': [3 / 5 / .9, .5 / 0.9, 1.9], - 'processing-loc': [4 / 5, 1 / 1, 1]}} + {'all': [5, 4, 1], + 'memory': [2, 1, 0], + 'erred': [0, 0, 1], + 'released': [1, 1, 0], + 'processing': [1, 2, 0], + 'queued': [1, 2, 0], + 'name': ['inc', 'add', 'dec'], + 'show-name': ['inc', 'add', 'dec'], + 'left': [0, 0, 1], + 'right': [0.9, 0.9, 1.9], + 'top': [0, -1, 0], + 'bottom': [-0.8, -1.8, -0.8], + 'color': ['#45BF6F', '#2E6C8E', '#440154'], + 'released-loc': [0.18, 0.225, 1.0], + 'memory-loc': [0.54, 0.45, 1.0], + 'erred-loc': [0.54, 0.45, 1.9], + 'processing-loc': [0.72, 0.9, 1.9], + 'queued-loc': [0.9, 1.35, 1.9], + 'done': ['3 / 5', '2 / 4', '1 / 1']} """ width = 0.9 names = sorted(msg["all"], key=msg["all"].get, reverse=True) @@ -100,19 +106,28 @@ def progress_quads(msg, nrows=8, ncols=3): d["memory-loc"] = [] d["erred-loc"] = [] d["processing-loc"] = [] + d["queued-loc"] = [] d["done"] = [] - for r, m, e, p, a, l in zip( - d["released"], d["memory"], d["erred"], d["processing"], d["all"], d["left"] + for r, m, e, p, q, a, l in zip( + d["released"], + d["memory"], + d["erred"], + d["processing"], + d["queued"], + d["all"], + d["left"], ): rl = width * r / a + l ml = width * (r + m) / a + l el = width * (r + m + e) / a + l pl = width * (p + r + m + e) / a + l + ql = width * (p + r + m + e + q) / a + l done = "%d / %d" % (r + m + e, a) d["released-loc"].append(rl) d["memory-loc"].append(ml) d["erred-loc"].append(el) d["processing-loc"].append(pl) + d["queued-loc"].append(ql) d["done"].append(done) return d From 1496abb3ba04e7e2acf77a312eddf213c5b3ee3c Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 13:39:43 -0600 Subject: [PATCH 04/98] improve reasonableness of task-state order Now task states on the dashboard are listed in the logical order that tasks transition through. --- distributed/dashboard/components/scheduler.py | 21 ++++++++++--------- distributed/tests/test_scheduler.py | 2 +- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/distributed/dashboard/components/scheduler.py b/distributed/dashboard/components/scheduler.py index 8aee47b26c9..7d5478ed4d7 100644 --- a/distributed/dashboard/components/scheduler.py +++ b/distributed/dashboard/components/scheduler.py @@ -3106,20 +3106,20 @@ def __init__(self, scheduler, **kwargs): @all
- Memory:  - @memory + Queued:  + @queued
- Erred:  - @erred + Processing:  + @processing
- Ready:  - @processing + Memory:  + @memory
- Queued:  - @queued + Erred:  + @erred
""", ) @@ -3166,9 +3166,10 @@ def update(self): self.root.title.text = ( "Progress -- total: %(all)s, " - "in-memory: %(memory)s, processing: %(processing)s, " - "queued: %(queued)s, " "waiting: %(waiting)s, " + "queued: %(queued)s, " + "processing: %(processing)s, " + "in-memory: %(memory)s, " "erred: %(erred)s" % totals ) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 125f09a8827..f1c6d0fce87 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -274,7 +274,7 @@ def big_data(size: int) -> str: return "x" * size roots = [ - big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(16) + big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(1600) ] passthrough = [delayed(slowidentity)(x) for x in roots] memory_consumed = [delayed(len)(x) for x in passthrough] From 74578652838d5f85611b52ef6a7454463d232170 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 15:28:43 -0600 Subject: [PATCH 05/98] Allow configurable oversaturation --- distributed/distributed-schema.yaml | 23 ++++++++++++++++ distributed/distributed.yaml | 1 + distributed/scheduler.py | 19 ++++++++++--- distributed/tests/test_scheduler.py | 41 +++++++++++++++++++++++++++-- 4 files changed, 79 insertions(+), 5 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 8d73b7df145..ca01bb7b79d 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -117,6 +117,29 @@ properties: description: | How frequently to balance worker loads + worker-oversaturation: + type: + - integer + - float + description: | + Controls how many extra root tasks are sent to workers (like a readahead). + + As an integer, this many _extra_ root tasks are sent to the worker, beyond that worker's + thread count. As a float, `worker-oversaturation * worker.nthreads` _extra_ tasks are sent + to the worker beyond its thread count. If `.inf`, all runnable tasks are immediately sent + to workers. + + Allowing oversaturation means a worker will start running a new root task as soon as + it completes the previous, even if there is a higher-priority downstream task to run. + This reduces worker idleness, by letting workers do something while waiting for further + instructions from the scheduler. + + This generally comes at the expense of increased memory usage. It leads to "wider" + (more breadth-first) execution of the graph. + + Compute-bound workloads benefit from increasing oversaturation. Memory-bound + workloads should generally leave `worker-oversaturation` at 0. + worker-ttl: type: - string diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index 3cf75a23298..bfabf140d43 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -22,6 +22,7 @@ distributed: events-log-length: 100000 work-stealing: True # workers should steal tasks from each other work-stealing-interval: 100ms # Callback time for work stealing + worker-oversaturation: 0 # Amount of extra root tasks to send to workers, as fixed count or fraction of nthreads. worker-ttl: "5 minutes" # like '60s'. Time to live for workers. They must heartbeat faster than this pickle: True # Is the scheduler allowed to deserialize arbitrary bytestrings preload: [] # Run custom modules with Scheduler diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 13ecb35d8f9..cfda49ceb99 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1301,6 +1301,7 @@ class SchedulerState: "MEMORY_REBALANCE_SENDER_MIN", "MEMORY_REBALANCE_RECIPIENT_MAX", "MEMORY_REBALANCE_HALF_GAP", + "WORKER_OVERSATURATION", } def __init__( @@ -1369,6 +1370,9 @@ def __init__( dask.config.get("distributed.worker.memory.rebalance.sender-recipient-gap") / 2.0 ) + self.WORKER_OVERSATURATION = dask.config.get( + "distributed.scheduler.worker-oversaturation" + ) self.transition_counter = 0 self._idle_transition_counter = 0 self.transition_counter_max = transition_counter_max @@ -1824,7 +1828,7 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: tg.last_worker_tasks_left -= 1 # Queue if worker is full to avoid root task overproduction. - if len(ws.processing) >= ws.nthreads: + if worker_saturated(ws, self.WORKER_OVERSATURATION): # TODO this should be a transition function instead. # But how do we get the `ws` into it? Recommendations on the scheduler can't take arguments. @@ -2534,7 +2538,7 @@ def transition_queued_processing(self, key, stimulus_id): # TODO other validation that this is still an appropriate worker? - if len(ws.processing) < ws.nthreads: + if not worker_saturated(ws, self.WORKER_OVERSATURATION): # If more important tasks already got scheduled, remain queued ts.queued_on = None @@ -7377,7 +7381,7 @@ def _remove_from_processing( state.release_resources(ts, ws) # If a slot has opened up for a queued task, schedule it. - if ws.queued and len(ws.processing) < ws.nthreads: + if ws.queued and not worker_saturated(ws, state.WORKER_OVERSATURATION): # TODO peek or pop? # What if multiple tasks complete on a worker in one transition cycle? Is that possible? # TODO should we only be scheduling 1 taks? Or N open threads? Is there a possible deadlock @@ -7773,6 +7777,15 @@ def heartbeat_interval(n: int) -> float: return n / 200 + 1 +def worker_saturated(ws: WorkerState, oversaturation_factor: int | float) -> bool: + if math.isinf(oversaturation_factor): + return False + nthreads = ws.nthreads + if isinstance(oversaturation_factor, float): + oversaturation_factor = math.floor(oversaturation_factor * nthreads) + return len(ws.processing) >= max(nthreads + oversaturation_factor, 1) + + class KilledWorker(Exception): def __init__(self, task: str, last_worker: WorkerState): super().__init__(task, last_worker) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index f1c6d0fce87..b753790742d 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -249,7 +249,7 @@ def random(**kwargs): @pytest.mark.slow @gen_cluster( client=True, - nthreads=[("127.0.0.1", 2)] * 2, + nthreads=[("", 2)] * 2, worker_kwargs={"memory_limit": "1.0GiB"}, timeout=3600, # TODO remove Worker=Nanny, @@ -274,7 +274,7 @@ def big_data(size: int) -> str: return "x" * size roots = [ - big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(1600) + big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(16) ] passthrough = [delayed(slowidentity)(x) for x in roots] memory_consumed = [delayed(len)(x) for x in passthrough] @@ -284,6 +284,43 @@ def big_data(size: int) -> str: await c.compute(final) +@pytest.mark.parametrize( + "oversaturation, expected_task_counts", + [ + (1.5, (5, 2)), + (1, (3, 2)), + (1.0, (4, 2)), + (0, (2, 1)), + (-1, (1, 1)), + (float("inf"), (7, 3)) + # ^ depends on root task assignment logic; ok if changes, just needs to add up to 10 + ], +) +def test_oversaturation_factor(oversaturation, expected_task_counts: tuple[int, int]): + @gen_cluster( + client=True, + nthreads=[("", 2), ("", 1)], + config={ + "distributed.scheduler.worker-oversaturation": oversaturation, + }, + ) + async def _test_oversaturation_factor(c, s, a, b): + event = Event() + fs = c.map(lambda _: event.wait(), range(10)) + while a.state.executing_count < min( + a.nthreads, expected_task_counts[0] + ) or b.state.executing_count < min(b.nthreads, expected_task_counts[1]): + await asyncio.sleep(0.01) + + assert len(a.state.tasks) == expected_task_counts[0] + assert len(b.state.tasks) == expected_task_counts[1] + + await event.set() + await c.gather(fs) + + _test_oversaturation_factor() + + @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) async def test_move_data_over_break_restrictions(client, s, a, b, c): [x] = await client.scatter([1], workers=b.address) From 67e9bd2a55f0eba70b91bb21a2d43083e084e8f4 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 15:40:28 -0600 Subject: [PATCH 06/98] Only support floats for `worker-oversaturation` Simpler, though I think basically just an int of 1 may be the most reasonable. --- distributed/distributed-schema.yaml | 17 +++++++---------- distributed/distributed.yaml | 2 +- distributed/scheduler.py | 8 ++++---- distributed/tests/test_scheduler.py | 5 ++--- 4 files changed, 14 insertions(+), 18 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index ca01bb7b79d..aa42c08972a 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -118,16 +118,12 @@ properties: How frequently to balance worker loads worker-oversaturation: - type: - - integer - - float + type: float description: | - Controls how many extra root tasks are sent to workers (like a readahead). + Controls how many extra root tasks are sent to workers (like a `readahead`). - As an integer, this many _extra_ root tasks are sent to the worker, beyond that worker's - thread count. As a float, `worker-oversaturation * worker.nthreads` _extra_ tasks are sent - to the worker beyond its thread count. If `.inf`, all runnable tasks are immediately sent - to workers. + `floor(worker-oversaturation * worker.nthreads)` _extra_ tasks are sent to the worker + beyond its thread count. If `.inf`, all runnable tasks are immediately sent to workers. Allowing oversaturation means a worker will start running a new root task as soon as it completes the previous, even if there is a higher-priority downstream task to run. @@ -137,8 +133,9 @@ properties: This generally comes at the expense of increased memory usage. It leads to "wider" (more breadth-first) execution of the graph. - Compute-bound workloads benefit from increasing oversaturation. Memory-bound - workloads should generally leave `worker-oversaturation` at 0. + Compute-bound workloads benefit from oversaturation. Memory-bound workloads should + generally leave `worker-oversaturation` at 0, though 0.25-0.5 could slightly improve + performance if ample memory is available. worker-ttl: type: diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index bfabf140d43..a37e1c34add 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -22,7 +22,7 @@ distributed: events-log-length: 100000 work-stealing: True # workers should steal tasks from each other work-stealing-interval: 100ms # Callback time for work stealing - worker-oversaturation: 0 # Amount of extra root tasks to send to workers, as fixed count or fraction of nthreads. + worker-oversaturation: 0.0 # Send this fraction of nthreads extra root tasks to workers worker-ttl: "5 minutes" # like '60s'. Time to live for workers. They must heartbeat faster than this pickle: True # Is the scheduler allowed to deserialize arbitrary bytestrings preload: [] # Run custom modules with Scheduler diff --git a/distributed/scheduler.py b/distributed/scheduler.py index cfda49ceb99..e47a159039f 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7777,13 +7777,13 @@ def heartbeat_interval(n: int) -> float: return n / 200 + 1 -def worker_saturated(ws: WorkerState, oversaturation_factor: int | float) -> bool: +def worker_saturated(ws: WorkerState, oversaturation_factor: float) -> bool: if math.isinf(oversaturation_factor): return False nthreads = ws.nthreads - if isinstance(oversaturation_factor, float): - oversaturation_factor = math.floor(oversaturation_factor * nthreads) - return len(ws.processing) >= max(nthreads + oversaturation_factor, 1) + return len(ws.processing) >= max( + nthreads + int(oversaturation_factor * nthreads), 1 + ) class KilledWorker(Exception): diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index b753790742d..f4900a67f55 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -288,10 +288,9 @@ def big_data(size: int) -> str: "oversaturation, expected_task_counts", [ (1.5, (5, 2)), - (1, (3, 2)), (1.0, (4, 2)), - (0, (2, 1)), - (-1, (1, 1)), + (0.0, (2, 1)), + (-1.0, (1, 1)), (float("inf"), (7, 3)) # ^ depends on root task assignment logic; ok if changes, just needs to add up to 10 ], From 2410a82a897dc45ea69e485256948e73265c608d Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 15:45:12 -0600 Subject: [PATCH 07/98] Push memory limits a little more in test --- distributed/tests/test_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index f4900a67f55..5dec6d8c5cb 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -274,7 +274,7 @@ def big_data(size: int) -> str: return "x" * size roots = [ - big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(16) + big_data(parse_bytes("350 MiB"), dask_key_name=f"root-{i}") for i in range(16) ] passthrough = [delayed(slowidentity)(x) for x in roots] memory_consumed = [delayed(len)(x) for x in passthrough] From 49d5ddd2ef74a76a7939d35058e122bf730094cf Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 19:25:40 -0600 Subject: [PATCH 08/98] Queued tasks on info pages --- distributed/http/templates/worker-table.html | 2 ++ distributed/http/templates/worker.html | 15 +++++++++++++++ 2 files changed, 17 insertions(+) diff --git a/distributed/http/templates/worker-table.html b/distributed/http/templates/worker-table.html index 87512ee3860..765d97133d7 100644 --- a/distributed/http/templates/worker-table.html +++ b/distributed/http/templates/worker-table.html @@ -6,6 +6,7 @@ Memory Memory use Occupancy + Queued Processing In-memory Services @@ -20,6 +21,7 @@ {{ format_bytes(ws.memory_limit) if ws.memory_limit is not None else "" }} {{ format_time(ws.occupancy) }} + {{ len(ws.queued) }} {{ len(ws.processing) }} {{ len(ws.has_what) }} {% if 'dashboard' in ws.services %} diff --git a/distributed/http/templates/worker.html b/distributed/http/templates/worker.html index 9c7608cb8c2..f5795248200 100644 --- a/distributed/http/templates/worker.html +++ b/distributed/http/templates/worker.html @@ -41,6 +41,21 @@

Processing

{% end %} +
+

Queued

+ + + + + + {% for ts in ws.queued.sorted() %} + + + + + {% end %} +
Task Priority
{{ts.key}} {{ts.priority }}
+
From b5469976903bb6b79a480712ef57ba7fefcd4571 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 22 Jul 2021 19:41:45 -0800 Subject: [PATCH 09/98] driveby: WIP color task graph by worker This is just a hack currently, but maybe it would actually be useful? --- distributed/dashboard/components/scheduler.py | 23 ++++++++++++++++++- distributed/diagnostics/graph_layout.py | 15 +++++++++--- 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/distributed/dashboard/components/scheduler.py b/distributed/dashboard/components/scheduler.py index 7d5478ed4d7..8b6fe0a640f 100644 --- a/distributed/dashboard/components/scheduler.py +++ b/distributed/dashboard/components/scheduler.py @@ -2100,6 +2100,13 @@ def __init__(self, scheduler, **kwargs): factors=["waiting", "processing", "memory", "released", "erred"], palette=["gray", "green", "red", "blue", "black"], ) + worker_colors = linear_cmap( + "worker", + Viridis11, # TODO larger cmap for more workers + low=0, + high=11, # TODO actually set this, and update when workers add/leave!!! + nan_color="black", + ) self.root = figure(title="Task Graph", **kwargs) self.subtitle = Title(text=" ", text_font_style="italic") @@ -2118,6 +2125,7 @@ def __init__(self, scheduler, **kwargs): x="x", y="y", size=10, + line_color=worker_colors, color=node_colors, source=self.node_source, view=node_view, @@ -2130,7 +2138,7 @@ def __init__(self, scheduler, **kwargs): hover = HoverTool( point_policy="follow_mouse", - tooltips="@name: @state", + tooltips="@name: @state @worker", renderers=[rect], ) tap = TapTool(callback=OpenURL(url="info/task/@key.html"), renderers=[rect]) @@ -2180,6 +2188,7 @@ def add_new_nodes_edges(self, new, new_edges, update=False): node_name = [] edge_x = [] edge_y = [] + worker = [] x = self.layout.x y = self.layout.y @@ -2197,6 +2206,11 @@ def add_new_nodes_edges(self, new, new_edges, update=False): node_y.append(yy) node_state.append(task.state) node_name.append(task.prefix.name) + ws = task.processing_on or ( + next(iter(task.who_has)) if task.who_has else None + ) + # TODO don't rely on worker name being int-like; use categorical cmap instead + worker.append(int(ws.name) if ws else None) for a, b in new_edges: try: @@ -2212,6 +2226,7 @@ def add_new_nodes_edges(self, new, new_edges, update=False): "name": node_name, "key": node_key, "visible": ["True"] * len(node_x), + "worker": worker, } edge = {"x": edge_x, "y": edge_y, "visible": ["True"] * len(edge_x)} @@ -2237,6 +2252,12 @@ def patch_updates(self): updates = [(i, c) for i, c in state_updates if i < n] self.node_source.patch({"state": updates}) + if self.layout.worker_updates: + worker_updates = self.layout.worker_updates + self.layout.worker_updates = [] + updates = [(i, c) for i, c in worker_updates if i < n] + self.node_source.patch({"worker": updates}) + if self.layout.visible_updates: updates = self.layout.visible_updates updates = [(i, c) for i, c in updates if i < n] diff --git a/distributed/diagnostics/graph_layout.py b/distributed/diagnostics/graph_layout.py index 88bdbeddebb..90b72f003bc 100644 --- a/distributed/diagnostics/graph_layout.py +++ b/distributed/diagnostics/graph_layout.py @@ -27,6 +27,7 @@ def __init__(self, scheduler): self.new = [] self.new_edges = [] self.state_updates = [] + self.worker_updates = [] self.visible_updates = [] self.visible_edge_updates = [] @@ -93,11 +94,18 @@ def update_graph( self.new_edges.append(edge) def transition(self, key, start, finish, *args, **kwargs): + task = self.scheduler.tasks[key] + idx = self.index[key] if finish != "forgotten": - self.state_updates.append((self.index[key], finish)) + self.state_updates.append((idx, finish)) + ws = ( + task.queued_on + or task.processing_on + or (next(iter(task.who_has)) if task.who_has else None) + ) + self.worker_updates.append((idx, int(ws.name) if ws else None)) else: - self.visible_updates.append((self.index[key], "False")) - task = self.scheduler.tasks[key] + self.visible_updates.append((idx, "False")) for dep in task.dependents: edge = (key, dep.key) self.visible_edge_updates.append((self.index_edge.pop(edge), "False")) @@ -124,6 +132,7 @@ def reset_index(self): self.new_edges = [] self.visible_updates = [] self.state_updates = [] + self.worker_updates = [] self.visible_edge_updates = [] self.index = {} From 2b44820c000e8c6a14b712adf5195add05f6bb89 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 20:06:32 -0600 Subject: [PATCH 10/98] Revert "driveby: WIP color task graph by worker" This reverts commit df11f719b59aad11f39a27ccae7b2fd4dfd9243a. --- distributed/dashboard/components/scheduler.py | 23 +------------------ distributed/diagnostics/graph_layout.py | 15 +++--------- 2 files changed, 4 insertions(+), 34 deletions(-) diff --git a/distributed/dashboard/components/scheduler.py b/distributed/dashboard/components/scheduler.py index 8b6fe0a640f..7d5478ed4d7 100644 --- a/distributed/dashboard/components/scheduler.py +++ b/distributed/dashboard/components/scheduler.py @@ -2100,13 +2100,6 @@ def __init__(self, scheduler, **kwargs): factors=["waiting", "processing", "memory", "released", "erred"], palette=["gray", "green", "red", "blue", "black"], ) - worker_colors = linear_cmap( - "worker", - Viridis11, # TODO larger cmap for more workers - low=0, - high=11, # TODO actually set this, and update when workers add/leave!!! - nan_color="black", - ) self.root = figure(title="Task Graph", **kwargs) self.subtitle = Title(text=" ", text_font_style="italic") @@ -2125,7 +2118,6 @@ def __init__(self, scheduler, **kwargs): x="x", y="y", size=10, - line_color=worker_colors, color=node_colors, source=self.node_source, view=node_view, @@ -2138,7 +2130,7 @@ def __init__(self, scheduler, **kwargs): hover = HoverTool( point_policy="follow_mouse", - tooltips="@name: @state @worker", + tooltips="@name: @state", renderers=[rect], ) tap = TapTool(callback=OpenURL(url="info/task/@key.html"), renderers=[rect]) @@ -2188,7 +2180,6 @@ def add_new_nodes_edges(self, new, new_edges, update=False): node_name = [] edge_x = [] edge_y = [] - worker = [] x = self.layout.x y = self.layout.y @@ -2206,11 +2197,6 @@ def add_new_nodes_edges(self, new, new_edges, update=False): node_y.append(yy) node_state.append(task.state) node_name.append(task.prefix.name) - ws = task.processing_on or ( - next(iter(task.who_has)) if task.who_has else None - ) - # TODO don't rely on worker name being int-like; use categorical cmap instead - worker.append(int(ws.name) if ws else None) for a, b in new_edges: try: @@ -2226,7 +2212,6 @@ def add_new_nodes_edges(self, new, new_edges, update=False): "name": node_name, "key": node_key, "visible": ["True"] * len(node_x), - "worker": worker, } edge = {"x": edge_x, "y": edge_y, "visible": ["True"] * len(edge_x)} @@ -2252,12 +2237,6 @@ def patch_updates(self): updates = [(i, c) for i, c in state_updates if i < n] self.node_source.patch({"state": updates}) - if self.layout.worker_updates: - worker_updates = self.layout.worker_updates - self.layout.worker_updates = [] - updates = [(i, c) for i, c in worker_updates if i < n] - self.node_source.patch({"worker": updates}) - if self.layout.visible_updates: updates = self.layout.visible_updates updates = [(i, c) for i, c in updates if i < n] diff --git a/distributed/diagnostics/graph_layout.py b/distributed/diagnostics/graph_layout.py index 90b72f003bc..88bdbeddebb 100644 --- a/distributed/diagnostics/graph_layout.py +++ b/distributed/diagnostics/graph_layout.py @@ -27,7 +27,6 @@ def __init__(self, scheduler): self.new = [] self.new_edges = [] self.state_updates = [] - self.worker_updates = [] self.visible_updates = [] self.visible_edge_updates = [] @@ -94,18 +93,11 @@ def update_graph( self.new_edges.append(edge) def transition(self, key, start, finish, *args, **kwargs): - task = self.scheduler.tasks[key] - idx = self.index[key] if finish != "forgotten": - self.state_updates.append((idx, finish)) - ws = ( - task.queued_on - or task.processing_on - or (next(iter(task.who_has)) if task.who_has else None) - ) - self.worker_updates.append((idx, int(ws.name) if ws else None)) + self.state_updates.append((self.index[key], finish)) else: - self.visible_updates.append((idx, "False")) + self.visible_updates.append((self.index[key], "False")) + task = self.scheduler.tasks[key] for dep in task.dependents: edge = (key, dep.key) self.visible_edge_updates.append((self.index_edge.pop(edge), "False")) @@ -132,7 +124,6 @@ def reset_index(self): self.new_edges = [] self.visible_updates = [] self.state_updates = [] - self.worker_updates = [] self.visible_edge_updates = [] self.index = {} From e494e874eb9f8a39a7c5f33781ec1010f6a59948 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 20:03:35 -0600 Subject: [PATCH 11/98] Queued tasks on graph --- distributed/dashboard/components/scheduler.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/distributed/dashboard/components/scheduler.py b/distributed/dashboard/components/scheduler.py index 7d5478ed4d7..8fc3b63cdea 100644 --- a/distributed/dashboard/components/scheduler.py +++ b/distributed/dashboard/components/scheduler.py @@ -2097,8 +2097,8 @@ def __init__(self, scheduler, **kwargs): node_colors = factor_cmap( "state", - factors=["waiting", "processing", "memory", "released", "erred"], - palette=["gray", "green", "red", "blue", "black"], + factors=["waiting", "queued", "processing", "memory", "released", "erred"], + palette=["gray", "yellow", "green", "red", "blue", "black"], ) self.root = figure(title="Task Graph", **kwargs) From ad417ed7f81fb66f39eb2f1f48eb351fec0dd98d Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 19:27:44 -0600 Subject: [PATCH 12/98] Redistribute queues when new worker joins --- distributed/collections.py | 39 +++++++++++++- distributed/scheduler.py | 81 +++++++++++++++++++++++++---- distributed/tests/test_scheduler.py | 15 ++++++ 3 files changed, 124 insertions(+), 11 deletions(-) diff --git a/distributed/collections.py b/distributed/collections.py index 4c978ebf0d7..07ef665e20b 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -1,6 +1,7 @@ from __future__ import annotations import heapq +import itertools import weakref from collections import OrderedDict, UserDict from collections.abc import Callable, Hashable, Iterator @@ -102,6 +103,41 @@ def pop(self) -> T: self._data.discard(value) return value + def popright(self) -> T: + "Remove and return one of the largest elements (not necessarily the largest)!" + if not self._data: + raise KeyError("popright from an empty set") + while True: + _, _, vref = self._heap.pop() + value = vref() + if value is not None and value in self._data: + self._data.discard(value) + return value + + def topk(self, k: int) -> Iterator[T]: + # TODO confirm big-O values here + "Iterator over the largest K elements. This is O(k*logn) for k < n // 2, O(n*logn) otherwise." + k = min(k, len(self)) + if k == 1: + yield self.peek() + elif k >= len(self) // 2: + return itertools.islice(self.sorted(), k) + else: + # FIXME though neat, with all the list mutation this is probably always slower than sorting inplace. + elems: list[tuple[Any, int, weakref.ref[T]]] = [] + try: + while len(elems) < k: + elem = heapq.heappop(self._heap) + value = elem[-1]() + if value is not None and value in self._data: + # NOTE: we're in a broken state during iteration, since the value exists + # in the set but not the heap. As with all Python iterators, mutating + # while iterating is undefined. + elems.append(elem) + yield value + finally: + self._heap = elems + self._heap + def __iter__(self) -> Iterator[T]: """Iterate over all elements. This is a O(n) operation which returns the elements in pseudo-random order. @@ -113,7 +149,8 @@ def sorted(self) -> Iterator[T]: elements in order, from smallest to largest according to the key and insertion order. """ - for _, _, vref in sorted(self._heap): + self._heap.sort() # A sorted list maintains the heap invariant + for _, _, vref in self._heap: value = vref() if value in self._data: yield value diff --git a/distributed/scheduler.py b/distributed/scheduler.py index e47a159039f..42018112f2e 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1828,7 +1828,7 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: tg.last_worker_tasks_left -= 1 # Queue if worker is full to avoid root task overproduction. - if worker_saturated(ws, self.WORKER_OVERSATURATION): + if task_slots_available(ws, self.WORKER_OVERSATURATION) <= 0: # TODO this should be a transition function instead. # But how do we get the `ws` into it? Recommendations on the scheduler can't take arguments. @@ -2538,7 +2538,7 @@ def transition_queued_processing(self, key, stimulus_id): # TODO other validation that this is still an appropriate worker? - if not worker_saturated(ws, self.WORKER_OVERSATURATION): + if task_slots_available(ws, self.WORKER_OVERSATURATION) > 0: # If more important tasks already got scheduled, remain queued ts.queued_on = None @@ -2946,9 +2946,71 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): ordering, so the recommendations are sorted by priority order here. """ ts: TaskState - tasks = [] - # TODO maintain global queue of tasks and reallocate them here - # FIXME queued tasks will not be assigned to a new worker at all!! + recommendations: dict[str, str] = {} + + # Redistribute the tasks between all worker queues. We bubble tasks off the back of the most-queued + # worker onto the front of the least-queued, and repeat this until we've accumulated enough tasks to + # put onto the new worker. This maintains the co-assignment of each worker's queue, minimizing the + # fragmentation of neighboring tasks. + # Note this does not rebalance all workers. It just rebalances the busiest workers, stealing just enough + # tasks to fill up the new worker. + # NOTE: this is probably going to be pretty slow for lots of queued tasks and/or lots of workers. + # Also unclear if this is even a good load-balancing strategy. + # TODO this is optimized for the add-worker case. Generalize for remove-worker as well. + # That would probably look like rebalancing all workers though. + if not math.isinf(self.WORKER_OVERSATURATION): + workers_with_queues: list[WorkerState] = sorted( + (wss for wss in self.workers.values() if wss.queued and wss is not ws), + key=lambda wss: len(wss.queued), + reverse=True, + ) + if workers_with_queues: + total_queued = sum(len(wss.queued) for wss in workers_with_queues) + target_qsize = int(total_queued / len(self.workers)) + moveable_tasks_so_far = 0 + last_q_tasks_to_move = 0 + i = 0 + # Go through workers with the largest queues until we've found enough workers to steal from + for i, wss in enumerate(workers_with_queues): + n_extra_tasks = len(wss.queued) - target_qsize + if n_extra_tasks <= 0: + break + moveable_tasks_so_far += n_extra_tasks + if moveable_tasks_so_far >= target_qsize: + last_q_tasks_to_move = n_extra_tasks - ( + moveable_tasks_so_far - target_qsize + ) + break + if last_q_tasks_to_move: + # Starting from the smallest, bubble tasks off the back of the queue and onto the front of the next-largest. + # At the end, bubble tasks onto the new worker's queue + while i >= 0: + src = workers_with_queues[i] + dest = workers_with_queues[i - 1] if i > 0 else ws + for _ in range(last_q_tasks_to_move): + # NOTE: `popright` is not exactly the highest element, but sorting would be too expensive. + # It's good enough, and in the common case the heap is sorted anyway (because elements are) + # inserted in sorted order by `decide_worker` + ts = src.queued.popright() + ts.queued_on = dest + dest.queued.add(ts) + + i -= 1 + last_q_tasks_to_move = target_qsize + + if ( + ws.queued + and (n := task_slots_available(ws, self.WORKER_OVERSATURATION)) > 0 + ): + # NOTE: reverse priority order, since recommendations are processed in LIFO order + for ts in reversed(list(ws.queued.topk(n))): + if self.validate: + assert ts.state == "queued" + assert ts.queued_on is ws, (ts.queued_on, ws) + assert ts.key not in recommendations, recommendations[ts.key] + recommendations[ts.key] = "processing" + + tasks: list[TaskState] = [] for ts in self.unrunnable: valid: set = self.valid_workers(ts) if valid is None or ws in valid: @@ -7381,7 +7443,7 @@ def _remove_from_processing( state.release_resources(ts, ws) # If a slot has opened up for a queued task, schedule it. - if ws.queued and not worker_saturated(ws, state.WORKER_OVERSATURATION): + if ws.queued and task_slots_available(ws, state.WORKER_OVERSATURATION) > 0: # TODO peek or pop? # What if multiple tasks complete on a worker in one transition cycle? Is that possible? # TODO should we only be scheduling 1 taks? Or N open threads? Is there a possible deadlock @@ -7777,13 +7839,12 @@ def heartbeat_interval(n: int) -> float: return n / 200 + 1 -def worker_saturated(ws: WorkerState, oversaturation_factor: float) -> bool: +def task_slots_available(ws: WorkerState, oversaturation_factor: float) -> int: + "Number of tasks that can be sent to this worker without oversaturating it" if math.isinf(oversaturation_factor): return False nthreads = ws.nthreads - return len(ws.processing) >= max( - nthreads + int(oversaturation_factor * nthreads), 1 - ) + return max(nthreads + int(oversaturation_factor * nthreads), 1) - len(ws.processing) class KilledWorker(Exception): diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 5dec6d8c5cb..b3d43f1266a 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -320,6 +320,21 @@ async def _test_oversaturation_factor(c, s, a, b): _test_oversaturation_factor() +@gen_cluster( + client=True, + nthreads=[("", 2)] * 2, + timeout=3600, # TODO remove + scheduler_kwargs=dict( # TODO remove + dashboard=True, + dashboard_address=":8787", + ), +) +async def test_queued_tasks_rebalance(c, s, a, b): + event = Event() + fs = c.map(lambda _: event.wait(), range(100)) + await c.gather(fs) + + @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) async def test_move_data_over_break_restrictions(client, s, a, b, c): [x] = await client.scatter([1], workers=b.address) From b4c698e4222cc7087a14cd7747feccd299935ea5 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 21:23:58 -0600 Subject: [PATCH 13/98] Fix task_slots_available when queuing disabled --- distributed/scheduler.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 42018112f2e..2b3ee7d7ba4 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1828,7 +1828,7 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: tg.last_worker_tasks_left -= 1 # Queue if worker is full to avoid root task overproduction. - if task_slots_available(ws, self.WORKER_OVERSATURATION) <= 0: + if worker_saturated(ws, self.WORKER_OVERSATURATION): # TODO this should be a transition function instead. # But how do we get the `ws` into it? Recommendations on the scheduler can't take arguments. @@ -2538,7 +2538,7 @@ def transition_queued_processing(self, key, stimulus_id): # TODO other validation that this is still an appropriate worker? - if task_slots_available(ws, self.WORKER_OVERSATURATION) > 0: + if not worker_saturated(ws, self.WORKER_OVERSATURATION): # If more important tasks already got scheduled, remain queued ts.queued_on = None @@ -7443,7 +7443,7 @@ def _remove_from_processing( state.release_resources(ts, ws) # If a slot has opened up for a queued task, schedule it. - if ws.queued and task_slots_available(ws, state.WORKER_OVERSATURATION) > 0: + if ws.queued and not worker_saturated(ws, state.WORKER_OVERSATURATION): # TODO peek or pop? # What if multiple tasks complete on a worker in one transition cycle? Is that possible? # TODO should we only be scheduling 1 taks? Or N open threads? Is there a possible deadlock @@ -7841,12 +7841,17 @@ def heartbeat_interval(n: int) -> float: def task_slots_available(ws: WorkerState, oversaturation_factor: float) -> int: "Number of tasks that can be sent to this worker without oversaturating it" - if math.isinf(oversaturation_factor): - return False + assert not math.isinf(oversaturation_factor) nthreads = ws.nthreads return max(nthreads + int(oversaturation_factor * nthreads), 1) - len(ws.processing) +def worker_saturated(ws: WorkerState, oversaturation_factor: float) -> bool: + if math.isinf(oversaturation_factor): + return False + return task_slots_available(ws, oversaturation_factor) <= 0 + + class KilledWorker(Exception): def __init__(self, task: str, last_worker: WorkerState): super().__init__(task, last_worker) From aa4e53128bc41084e4b6d970cb1aec17bc8a3f36 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 17 Jun 2022 23:00:46 -0600 Subject: [PATCH 14/98] Fix co-assignment logic to consider queued tasks When there were multiple root task groups, we were just re-using the last worker for every batch because it had nothing processing on it. Unintentionally this also fixes https://github.com/dask/distributed/issues/6597 in some cases (because the first task goes to processing, but we measure queued, so we pick the same worker for both task groups) --- distributed/scheduler.py | 17 +++++++++++++++-- distributed/tests/test_scheduler.py | 29 +++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 2b3ee7d7ba4..ae5143cf961 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1813,10 +1813,23 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: 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 + + # We just pick the worker with the shortest queue (or if queuing is disabled, + # the fewest processing tasks). We've already decided dependencies are unimportant, + # so we don't care to schedule near them. + backlog = operator.attrgetter( + "processing" if math.isinf(self.WORKER_OVERSATURATION) else "queued" + ) ws = min( - (self.idle or self.workers).values(), - key=partial(self.worker_objective, ts), + self.workers.values(), key=lambda ws: len(backlog(ws)) / ws.nthreads ) + if self.validate: + assert ws is not tg.last_worker, ( + f"Colocation reused worker {ws} for {tg}, " + f"idle: {list(self.idle.values())}, " + f"workers: {list(self.workers.values())}" + ) + tg.last_worker_tasks_left = math.floor( (len(tg) / self.total_nthreads) * ws.nthreads ) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index b3d43f1266a..cc16309678a 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -320,6 +320,35 @@ async def _test_oversaturation_factor(c, s, a, b): _test_oversaturation_factor() +@pytest.mark.parametrize( + "saturation_factor", + [ + 0.0, + 1.0, + pytest.param( + float("inf"), + marks=pytest.mark.skip("https://github.com/dask/distributed/issues/6597"), + ), + ], +) +@gen_cluster( + client=True, + nthreads=[("", 2), ("", 1)], +) +async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor): + s.WORKER_OVERSATURATION = saturation_factor + xs = [delayed(i, name=f"x-{i}") for i in range(9)] + ys = [delayed(i, name=f"y-{i}") for i in range(9)] + zs = [x + y for x, y in zip(xs, ys)] + + await c.gather(c.compute(zs)) + + assert not a.incoming_transfer_log, [l["keys"] for l in a.incoming_transfer_log] + assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] + assert len(a.tasks) == 18 + assert len(b.tasks) == 9 + + @gen_cluster( client=True, nthreads=[("", 2)] * 2, From b514e84c5295c6ee4b93cece07fb0e653dea3052 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Sun, 19 Jun 2022 23:28:39 -0600 Subject: [PATCH 15/98] Revert "unused: `OrderedSet` collection" This reverts commit fdd5fd9e327d3c75f0e676c7d940b95119d34687. --- distributed/collections.py | 116 +------------------------------------ 1 file changed, 2 insertions(+), 114 deletions(-) diff --git a/distributed/collections.py b/distributed/collections.py index 07ef665e20b..b79e0cd6c08 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -5,14 +5,8 @@ import weakref from collections import OrderedDict, UserDict from collections.abc import Callable, Hashable, Iterator -from typing import ( # TODO move to collections.abc (requires Python >=3.9) - AbstractSet, - Any, - Iterable, - MutableSet, - TypeVar, - cast, -) +from typing import MutableSet # TODO move to collections.abc (requires Python >=3.9) +from typing import Any, TypeVar, cast T = TypeVar("T", bound=Hashable) @@ -158,109 +152,3 @@ def sorted(self) -> Iterator[T]: def clear(self) -> None: self._data.clear() self._heap.clear() - - -class OrderedSet(MutableSet[T]): - """ - A insertion-ordered set. - - All operations are O(1) complexity. - - Equality tests between OrderedSet objects are order-sensitive. Equality tests - between OrderedSet objects and other AbstractSet objects are order-insensitive like - regular sets. - """ - - __slots__ = ("_data",) - _data: dict[T, None] - - def __init__(self, iterable: Iterable[T] | None = None) -> None: - if iterable: - self._data = dict.fromkeys(iterable) - else: - self._data = {} - - def add(self, value: T) -> None: - self._data[value] = None - # NOTE: updating an already-existing item in a dict does not change iteration order - - def discard(self, value: T) -> None: - self._data.pop(value, None) - - def clear(self) -> None: - self._data.clear() - - def copy(self) -> OrderedSet[T]: - new = type(self)() - new._data = self._data.copy() - return new - - def pop(self) -> T: - "Remove and return the last-inserted item" - if not self._data: - raise KeyError("pop on an empty set") - return self._data.popitem()[0] - - def popleft(self) -> T: - "Remove and return the first-inserted item" - if not self._data: - raise KeyError("popleft on an empty set") - first = next(iter(self._data)) - self._data.pop(first) - return first - - def peek(self) -> T: - if not self._data: - raise KeyError("peek into empty set") - return next(reversed(self._data)) - - def peekleft(self) -> T: - if not self._data: - raise KeyError("peekleft into empty set") - return next(iter(self._data)) - - def rotate(self, n=1) -> None: - """ - Rotate the OrderedSet ``n`` steps to the right. - - Note that each rotation is an O(1) operation, so the time-complexity - is equivalent to ``n``. - """ - if n == 0: - return - if n < 0: - raise ValueError(f"{type(self).__name__} can only be rotated to the right") - n = n % len(self) - for _ in range(n): - self.add(self.popleft()) - - def update(self, iterable: Iterable[T]) -> None: - for x in iterable: - self._data[x] = None - - def __repr__(self) -> str: - return f"{type(self).__name__}({', '.join(map(str, self))})>" - - def __contains__(self, value: object) -> bool: - return value in self._data - - def __len__(self) -> int: - return len(self._data) - - def __iter__(self) -> Iterator[T]: - """Iterate over all elements in insertion order.""" - return iter(self._data) - - def __reverse__(self) -> Iterator[T]: - """Iterate over all elements in reverse insertion order.""" - return reversed(self._data) - - def __eq__(self, other: object) -> bool: - if isinstance(other, type(self)): - return len(other._data) == len(self._data) and all( - a == b for a, b in zip(self._data, other._data) - ) - if isinstance(other, AbstractSet): - return self._data.keys() == other - - return NotImplemented From 1835a89349281cdea233e3632f77bcbc13c739fa Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Sun, 19 Jun 2022 23:37:23 -0600 Subject: [PATCH 16/98] Fix potential stale worker use in decide_worker --- distributed/scheduler.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index ae5143cf961..3848334be22 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1811,7 +1811,9 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: ): ws = tg.last_worker - if not (ws and tg.last_worker_tasks_left and ws.address in self.workers): + if not ( + ws and tg.last_worker_tasks_left and self.workers.get(ws.address) is ws + ): # Last-used worker is full or unknown; pick a new worker for the next few tasks # We just pick the worker with the shortest queue (or if queuing is disabled, From db42c2262d271bf79d6644928d269f89f96d0589 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 00:26:34 -0600 Subject: [PATCH 17/98] WIP identify root task families 1. The family metric itself is flawed. Added linear chain traversal, but it's still not good. The maxsize is problematic and probably the wrong way to think about it? a) there's quite likely no maxsize parameter that will ever be right, because you could always have multiple independent crazy substructures that are each maxsize+1. b) even when every task would be in the same family because they're all interconnected, there's still benefit to scheduling subsequent things together, even if you do partition. Minimizing priority partitions is always what you want. Maybe there's something where maxsize is not a hard cutoff, but a cutoff for where to split up interconnected structures? 2. Families probably need to be data structures? When a task completes, you'd like to know if it belongs to a family that actually has more tasks to run on that worker, vs the task just happens to look like it belongs to a family but was never scheduled as a rootish task. Overall I like the family structure for scheduling up/down scaling, but figuring out how to identify them is tricky. Partitioning priority order is great because it totally avoids this problem, of course at the expense of scaling. Can we combine priority and graph structure to identify isolated families when reasonable, partition on priority when not? --- distributed/http/templates/worker-table.html | 2 - distributed/http/templates/worker.html | 15 - distributed/scheduler.py | 451 +++++++++--------- distributed/tests/test_scheduler.py | 10 +- .../widgets/templates/worker_state.html.j2 | 1 - 5 files changed, 245 insertions(+), 234 deletions(-) diff --git a/distributed/http/templates/worker-table.html b/distributed/http/templates/worker-table.html index 765d97133d7..87512ee3860 100644 --- a/distributed/http/templates/worker-table.html +++ b/distributed/http/templates/worker-table.html @@ -6,7 +6,6 @@ Memory Memory use Occupancy - Queued Processing In-memory Services @@ -21,7 +20,6 @@ {{ format_bytes(ws.memory_limit) if ws.memory_limit is not None else "" }} {{ format_time(ws.occupancy) }} - {{ len(ws.queued) }} {{ len(ws.processing) }} {{ len(ws.has_what) }} {% if 'dashboard' in ws.services %} diff --git a/distributed/http/templates/worker.html b/distributed/http/templates/worker.html index f5795248200..9c7608cb8c2 100644 --- a/distributed/http/templates/worker.html +++ b/distributed/http/templates/worker.html @@ -41,21 +41,6 @@

Processing

{% end %} -
-

Queued

- - - - - - {% for ts in ws.queued.sorted() %} - - - - - {% end %} -
Task Priority
{{ts.key}} {{ts.priority }}
-
diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 3848334be22..eb220bb9d3c 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -16,7 +16,7 @@ import uuid import warnings import weakref -from collections import defaultdict, deque +from collections import Counter, defaultdict, deque from collections.abc import ( Callable, Collection, @@ -449,10 +449,6 @@ class WorkerState: #: been running. executing: dict[TaskState, float] - #: Tasks queued to _potentially_ run on this worker in the future, ordered by priority. - #: The queuing is scheduler-side only; the worker is unaware of these tasks. - queued: HeapSet[TaskState] - #: 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. @@ -507,7 +503,6 @@ def __init__( self.processing = {} self.long_running = set() self.executing = {} - self.queued = HeapSet(key=operator.attrgetter("priority")) self.resources = {} self.used_resources = {} self.extra = extra or {} @@ -577,8 +572,7 @@ def __repr__(self) -> str: f"" + f"processing: {len(self.processing)}>" ) def _repr_html_(self): @@ -588,7 +582,6 @@ def _repr_html_(self): status=self.status.name, has_what=self.has_what, processing=self.processing, - queued=self.queued, ) def identity(self) -> dict[str, Any]: @@ -808,14 +801,6 @@ class TaskGroup: #: 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 @@ -835,8 +820,6 @@ def __init__(self, name: str): 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 @@ -988,10 +971,6 @@ class TaskState: #: it. This attribute is kept in sync with :attr:`WorkerState.processing`. processing_on: WorkerState | None - #: If this task is in the "queued" state, which worker is currently queued - #: it. This attribute is kept in sync with :attr:`WorkerState.queued`. - queued_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 @@ -1113,7 +1092,6 @@ def __init__(self, key: str, run_spec: object): self.waiters = set() self.who_has = set() self.processing_on = None - self.queued_on = None self.has_lost_dependencies = False self.host_restrictions = None # type: ignore self.worker_restrictions = None # type: ignore @@ -1251,6 +1229,8 @@ class SchedulerState: Tasks currently known to the scheduler * **unrunnable:** ``{TaskState}`` Tasks in the "no-worker" state + * **queued:** ``HeapSet[TaskState]`` + Tasks in the "queued" state, ordered by priority * **workers:** ``{worker key: WorkerState}`` Workers currently connected to the scheduler @@ -1277,6 +1257,7 @@ class SchedulerState: "host_info", "idle", "n_tasks", + "queued", "resources", "saturated", "running", @@ -1313,6 +1294,7 @@ def __init__( resources: dict, tasks: dict, unrunnable: set, + queued: HeapSet[TaskState], validate: bool, plugins: Iterable[SchedulerPlugin] = (), transition_counter_max: int | Literal[False] = False, @@ -1342,6 +1324,7 @@ def __init__( self.total_nthreads = 0 self.total_occupancy = 0.0 self.unknown_durations: dict[str, set[TaskState]] = {} + self.queued = queued self.unrunnable = unrunnable self.validate = validate self.workers = workers @@ -1388,6 +1371,7 @@ def __pdict__(self): "resources": self.resources, "saturated": self.saturated, "unrunnable": self.unrunnable, + "queued": self.queued, "n_tasks": self.n_tasks, "unknown_durations": self.unknown_durations, "validate": self.validate, @@ -1773,7 +1757,9 @@ def transition_no_worker_memory( pdb.set_trace() raise - def decide_worker(self, ts: TaskState) -> WorkerState | None: + def decide_worker( + self, ts: TaskState, recommendations: dict[str, str] + ) -> WorkerState | None: """ Decide on a worker for task *ts*. Return a WorkerState. @@ -1803,65 +1789,39 @@ def decide_worker(self, ts: TaskState) -> WorkerState | None: # Group is larger than cluster with few dependencies? # Minimize future data transfers. + # TODO update metric to involve families in some way? Most likely not. 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 self.workers.get(ws.address) is ws - ): - # Last-used worker is full or unknown; pick a new worker for the next few tasks - - # We just pick the worker with the shortest queue (or if queuing is disabled, - # the fewest processing tasks). We've already decided dependencies are unimportant, - # so we don't care to schedule near them. - backlog = operator.attrgetter( - "processing" if math.isinf(self.WORKER_OVERSATURATION) else "queued" - ) - ws = min( - self.workers.values(), key=lambda ws: len(backlog(ws)) / ws.nthreads - ) - if self.validate: - assert ws is not tg.last_worker, ( - f"Colocation reused worker {ws} for {tg}, " - f"idle: {list(self.idle.values())}, " - f"workers: {list(self.workers.values())}" - ) - - 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 - - # Queue if worker is full to avoid root task overproduction. - if worker_saturated(ws, self.WORKER_OVERSATURATION): - # TODO this should be a transition function instead. - # But how do we get the `ws` into it? Recommendations on the scheduler can't take arguments. - + # TODO what max family size cutoff to use? + if ws := family_worker(ts, self.total_nthreads): + # Use the worker where the majority of other tasks are already assigned. + # Even if that worker is already saturated, we let more tasks run on it, + # because all those tasks must be in memory at once anyway to run the + # downstream task. + return ws + + # First task in this family, or such a huge family it doesn't matter where it runs? + # Pick the least busy worker. + if not self.idle: + # If all workers are busy, this family gets queued for later. if self.validate: - assert not ts.queued_on, ts.queued_on - assert ts not in ws.queued - - # TODO maintain global queue of tasks as well for newly arriving workers to use? - # QUESTION could `queued` be an OrderedSet instead of a HeapSet, giving us O(1) - # operations instead of O(logn)? Reasoning is that we're always inserting elements - # in priority order anyway. - # This wouldn't work in the case that a batch of lower-priority root tasks becomes - # ready before a batch of higher-priority root tasks. - ws.queued.add(ts) - ts.queued_on = ws - ts.state = "queued" + assert ts.key not in recommendations, (ts, recommendations[ts.key]) + if ts.state != "queued": + recommendations[ts.key] = "queued" return None + ws = min( + self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads + ) + if self.validate: + assert not worker_saturated(ws, self.WORKER_OVERSATURATION), ( + ws, + task_slots_available(ws, self.WORKER_OVERSATURATION), + ) return ws if ts.dependencies or valid_workers is not None: @@ -1910,12 +1870,11 @@ def transition_waiting_processing(self, key, stimulus_id): assert not ts.who_has assert not ts.exception_blame assert not ts.processing_on - assert not ts.queued_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) + ws = self.decide_worker(ts, recommendations) if ws is None: return recommendations, client_msgs, worker_msgs worker = ws.address @@ -1962,7 +1921,6 @@ def transition_waiting_memory( if self.validate: assert not ts.processing_on - assert not ts.queued_on assert ts.waiting_on assert ts.state == "waiting" @@ -1979,7 +1937,6 @@ def transition_waiting_memory( if self.validate: assert not ts.processing_on - assert not ts.queued_on assert not ts.waiting_on assert ts.who_has @@ -2082,18 +2039,17 @@ def transition_processing_memory( if self.validate: assert not ts.processing_on - assert not ts.queued_on assert not ts.waiting_on - processing_recs = { - k: r for k, r in recommendations.items() if r == "processing" - } - assert list(processing_recs) == ( - sr := sorted( - processing_recs, - key=lambda k: self.tasks[k].priority, - reverse=True, - ) - ), (list(processing_recs), sr) + # processing_recs = { + # k: r for k, r in recommendations.items() if r == "processing" + # } + # assert list(processing_recs) == ( + # sr := sorted( + # processing_recs, + # key=lambda k: self.tasks[k].priority, + # reverse=True, + # ) + # ), (list(processing_recs), sr) return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2116,7 +2072,6 @@ def transition_memory_released(self, key, stimulus_id, safe: bool = False): if self.validate: assert not ts.waiting_on assert not ts.processing_on - assert not ts.queued_on if safe: assert not ts.waiters @@ -2278,7 +2233,6 @@ def transition_waiting_released(self, key, stimulus_id): if self.validate: assert not ts.who_has assert not ts.processing_on - assert not ts.queued_on dts: TaskState for dts in ts.dependencies: @@ -2347,7 +2301,6 @@ def transition_processing_released(self, key, stimulus_id): if self.validate: assert not ts.processing_on - assert not ts.queued_on return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2439,7 +2392,6 @@ def transition_processing_erred( if self.validate: assert not ts.processing_on - assert not ts.queued_on return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2480,23 +2432,54 @@ def transition_no_worker_released(self, key, stimulus_id): pdb.set_trace() raise - def transition_queued_released(self, key, stimulus_id): + def transition_waiting_queued(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] recommendations: dict = {} client_msgs: dict = {} worker_msgs: dict = {} - # TODO allow `remove_worker` to clear `queued_on` and `ws.queued` eagerly; it's more efficient. - ws = ts.queued_on - assert ws + if self.validate: + assert ts not in self.queued + # Task should have gone straight to processing if its family already had a worker + assert family_worker(ts, self.total_nthreads) is None, ( + key, + family(ts, self.total_nthreads), + ) + assert not self.idle, (ts, self.idle) + # Copied from `transition_waiting_processing` + 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) + + ts.state = "queued" + self.queued.add(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_queued_released(self, key, stimulus_id): + try: + ts: TaskState = self.tasks[key] + recommendations: dict = {} + client_msgs: dict = {} + worker_msgs: dict = {} if self.validate: - assert ts in ws.queued + assert ts in self.queued assert not ts.processing_on - ws.queued.remove(ts) - ts.queued_on = None + self.queued.remove(ts) # TODO copied from `transition_processing_released`; factor out into helper function ts.state = "released" @@ -2504,7 +2487,6 @@ def transition_queued_released(self, key, stimulus_id): if ts.has_lost_dependencies: recommendations[key] = "forgotten" elif ts.waiters or ts.who_wants: - # TODO rescheduling of queued root tasks may be poor. recommendations[key] = "waiting" if recommendations.get(key) != "waiting": @@ -2531,17 +2513,9 @@ def transition_queued_processing(self, key, stimulus_id): client_msgs: dict = {} worker_msgs: dict = {} - ws = ts.queued_on - assert ws - # TODO should this be a graceful transition to released? I think `remove_worker` - # makes it such that this should never happen. - assert ( - self.workers[ws.address] is ws - ), f"Task {ts} queued on stale worker {ws}" - if self.validate: assert not ts.actor, "Actors can't be queued wat" - assert ts in ws.queued + assert ts in self.queued # Copied from `transition_waiting_processing` assert not ts.processing_on assert not ts.waiting_on @@ -2551,13 +2525,9 @@ def transition_queued_processing(self, key, stimulus_id): assert ts not in self.unrunnable assert all(dts.who_has for dts in ts.dependencies) - # TODO other validation that this is still an appropriate worker? - - if not worker_saturated(ws, self.WORKER_OVERSATURATION): - # If more important tasks already got scheduled, remain queued - - ts.queued_on = None - ws.queued.remove(ts) + # NOTE: if all workers are now saturated and this task shouldn't actually run, `ws` is be None. + if ws := self.decide_worker(ts, recommendations): + self.queued.remove(ts) # TODO Copied from `transition_waiting_processing`; factor out into helper function self._set_duration_estimate(ts, ws) ts.processing_on = ws @@ -2573,6 +2543,8 @@ def transition_queued_processing(self, key, stimulus_id): worker_msgs[ws.address] = [_task_to_msg(self, ts)] + if self.validate: + assert not recommendations, recommendations return recommendations, client_msgs, worker_msgs except Exception as e: logger.exception(e) @@ -2605,7 +2577,6 @@ def transition_memory_forgotten(self, key, stimulus_id): if self.validate: assert ts.state == "memory" assert not ts.processing_on - assert not ts.queued_on assert not ts.waiting_on if not ts.run_spec: # It's ok to forget a pure data task @@ -2648,7 +2619,7 @@ def transition_released_forgotten(self, key, stimulus_id): assert ts.state in ("released", "erred") assert not ts.who_has assert not ts.processing_on - assert not ts.queued_on + assert ts not in self.queued assert not ts.waiting_on, (ts, ts.waiting_on) if not ts.run_spec: # It's ok to forget a pure data task @@ -2688,6 +2659,7 @@ def transition_released_forgotten(self, key, stimulus_id): ("released", "waiting"): transition_released_waiting, ("waiting", "released"): transition_waiting_released, ("waiting", "processing"): transition_waiting_processing, + ("waiting", "queued"): transition_waiting_queued, ("waiting", "memory"): transition_waiting_memory, ("queued", "released"): transition_queued_released, ("queued", "processing"): transition_queued_processing, @@ -2746,7 +2718,7 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): 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. + large enough. TODO update This is useful for load balancing and adaptivity. """ @@ -2761,12 +2733,13 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): idle = self.idle saturated = self.saturated - if p < nc or occ < nc * avg / 2: + if not worker_saturated(ws, self.WORKER_OVERSATURATION): idle[ws.address] = ws saturated.discard(ws) else: idle.pop(ws.address, None) + # TODO do we still want any of this? if p > nc: pending: float = occ * (p - nc) / (p * nc) if 0.4 < pending > 1.9 * avg: @@ -2963,77 +2936,45 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): ts: TaskState recommendations: dict[str, str] = {} - # Redistribute the tasks between all worker queues. We bubble tasks off the back of the most-queued - # worker onto the front of the least-queued, and repeat this until we've accumulated enough tasks to - # put onto the new worker. This maintains the co-assignment of each worker's queue, minimizing the - # fragmentation of neighboring tasks. - # Note this does not rebalance all workers. It just rebalances the busiest workers, stealing just enough - # tasks to fill up the new worker. - # NOTE: this is probably going to be pretty slow for lots of queued tasks and/or lots of workers. - # Also unclear if this is even a good load-balancing strategy. - # TODO this is optimized for the add-worker case. Generalize for remove-worker as well. - # That would probably look like rebalancing all workers though. - if not math.isinf(self.WORKER_OVERSATURATION): - workers_with_queues: list[WorkerState] = sorted( - (wss for wss in self.workers.values() if wss.queued and wss is not ws), - key=lambda wss: len(wss.queued), - reverse=True, - ) - if workers_with_queues: - total_queued = sum(len(wss.queued) for wss in workers_with_queues) - target_qsize = int(total_queued / len(self.workers)) - moveable_tasks_so_far = 0 - last_q_tasks_to_move = 0 - i = 0 - # Go through workers with the largest queues until we've found enough workers to steal from - for i, wss in enumerate(workers_with_queues): - n_extra_tasks = len(wss.queued) - target_qsize - if n_extra_tasks <= 0: - break - moveable_tasks_so_far += n_extra_tasks - if moveable_tasks_so_far >= target_qsize: - last_q_tasks_to_move = n_extra_tasks - ( - moveable_tasks_so_far - target_qsize - ) - break - if last_q_tasks_to_move: - # Starting from the smallest, bubble tasks off the back of the queue and onto the front of the next-largest. - # At the end, bubble tasks onto the new worker's queue - while i >= 0: - src = workers_with_queues[i] - dest = workers_with_queues[i - 1] if i > 0 else ws - for _ in range(last_q_tasks_to_move): - # NOTE: `popright` is not exactly the highest element, but sorting would be too expensive. - # It's good enough, and in the common case the heap is sorted anyway (because elements are) - # inserted in sorted order by `decide_worker` - ts = src.queued.popright() - ts.queued_on = dest - dest.queued.add(ts) - - i -= 1 - last_q_tasks_to_move = target_qsize - - if ( - ws.queued - and (n := task_slots_available(ws, self.WORKER_OVERSATURATION)) > 0 + if not math.isinf(self.WORKER_OVERSATURATION) and self.queued: + for qts in self.queued.topk( + slots := task_slots_available(ws, self.WORKER_OVERSATURATION) ): - # NOTE: reverse priority order, since recommendations are processed in LIFO order - for ts in reversed(list(ws.queued.topk(n))): + if self.validate: + assert qts.state == "queued" + + # Schedule the entire family + for fts in family(qts, self.total_nthreads) or (qts,): + # NOTE: `family` is not priority-ordered, but this is superfluous + # because all tasks in the family will get scheduled anyway + # regardless of order. if self.validate: - assert ts.state == "queued" - assert ts.queued_on is ws, (ts.queued_on, ws) - assert ts.key not in recommendations, recommendations[ts.key] - recommendations[ts.key] = "processing" + # If any tasks in a family are running or have run, no other tasks + # in that family should have been queued---they'd go straight to `processing`. + assert fts.state in ("waiting", "queued"), ( + fts, + family(qts, self.total_nthreads), + ) + slots -= 1 # We allocate slots for waiting tasks as well + if fts.state == "queued": + recommendations[fts.key] = "processing" + if slots <= 0: + break - tasks: list[TaskState] = [] + now_runnable: list[TaskState] = [] for ts in self.unrunnable: valid: set = self.valid_workers(ts) if valid is None or ws in valid: - tasks.append(ts) + now_runnable.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} + now_runnable.sort(key=operator.attrgetter("priority"), reverse=True) + for ts in now_runnable: + # TODO queued tasks will take precedence over newly-runnable tasks + # since waiting->processing will put them at the end of the transitions + # dict (since it's a `popitem` then `update`). Is that okay? + recommendations[ts.key] = "waiting" + return recommendations class Scheduler(SchedulerState, ServerNode): @@ -3241,6 +3182,7 @@ def __init__( self._last_client = None self._last_time = 0 unrunnable = set() + queued: HeapSet[TaskState] = HeapSet(key=operator.attrgetter("priority")) self.datasets = {} @@ -3377,6 +3319,7 @@ def __init__( resources=resources, tasks=tasks, unrunnable=unrunnable, + queued=queued, validate=validate, plugins=plugins, transition_counter_max=transition_counter_max, @@ -4513,10 +4456,6 @@ async def remove_worker( else: # pure data recommendations[ts.key] = "forgotten" - for ts in ws.queued.sorted(): - recommendations[ts.key] = "released" - # ws.queued.clear() # TODO more performant - self.transitions(recommendations, stimulus_id=stimulus_id) for plugin in list(self.plugins.values()): @@ -4635,17 +4574,17 @@ def validate_released(self, key): assert not ts.waiting_on assert not ts.who_has assert not ts.processing_on - assert not ts.queued_on assert not any([ts in dts.waiters for dts in ts.dependencies]) assert ts not in self.unrunnable + assert ts not in self.queued 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 not ts.queued_on assert ts not in self.unrunnable + assert ts not in self.queued 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) @@ -4654,25 +4593,27 @@ def validate_waiting(self, key): def validate_queued(self, key): ts: TaskState = self.tasks[key] dts: TaskState + assert ts in self.queued assert not ts.waiting_on - ws = ts.queued_on - assert ws - assert self.workers.get(ws.address) is ws, f"{ts} queued on stale worker {ws}" - assert ts in ws.queued assert not ts.who_has assert not ts.processing_on for dts in ts.dependencies: assert dts.who_has assert ts in dts.waiters + assert family_worker(ts, self.total_nthreads) is None, ( + key, + family(ts, self.total_nthreads), + ) def validate_processing(self, key): ts: TaskState = self.tasks[key] dts: TaskState assert not ts.waiting_on - ws: WorkerState = ts.processing_on + ws = ts.processing_on assert ws assert ts in ws.processing assert not ts.who_has + assert ts not in self.queued for dts in ts.dependencies: assert dts.who_has assert ts in dts.waiters @@ -4683,12 +4624,12 @@ def validate_memory(self, key): assert ts.who_has assert bool(ts in self.replicated_tasks) == (len(ts.who_has) > 1) assert not ts.processing_on - assert not ts.queued_on assert not ts.waiting_on assert ts not in self.unrunnable + assert ts not in self.queued for dts in ts.dependents: assert (dts in ts.waiters) == ( - dts.state in ("waiting", "processing", "no-worker") + dts.state in ("waiting", "queued", "processing", "no-worker") ) assert ts not in dts.waiting_on @@ -4698,8 +4639,8 @@ def validate_no_worker(self, key): assert not ts.waiting_on assert ts in self.unrunnable assert not ts.processing_on - assert not ts.queued_on assert not ts.who_has + assert ts not in self.queued for dts in ts.dependencies: assert dts.who_has @@ -4707,6 +4648,7 @@ def validate_erred(self, key): ts: TaskState = self.tasks[key] assert ts.exception_blame assert not ts.who_has + assert ts not in self.queued def validate_key(self, key, ts: TaskState = None): try: @@ -7442,7 +7384,7 @@ def _remove_from_processing( assert ws ts.processing_on = None - if ws.address not in state.workers: # may have been removed + if state.workers.get(ws.address) is not ws: # may have been removed return None duration = ws.processing.pop(ts) @@ -7457,18 +7399,50 @@ def _remove_from_processing( state.check_idle_saturated(ws) state.release_resources(ts, ws) - # If a slot has opened up for a queued task, schedule it. - if ws.queued and not worker_saturated(ws, state.WORKER_OVERSATURATION): - # TODO peek or pop? - # What if multiple tasks complete on a worker in one transition cycle? Is that possible? - # TODO should we only be scheduling 1 taks? Or N open threads? Is there a possible deadlock - # where tasks remain queued on a worker forever? - qts = ws.queued.peek() + # If a slot has opened up for a queued task, schedule it and its whole family. + if ( + state.queued + and task_slots_available(ws, state.WORKER_OVERSATURATION) > 0 + # TODO: leaving room causes deadlock when `ts` is a downstream, non-rootish task + # that just happens to meet the family criteria (and some of the family is waiting + # on queued tasks). + # But without this, we could have overproduction when some family members have deps + # and others don't (since we'll let a new family come in and start on the worker + # before the current family is done). + # and sum( + # fts.state == "waiting" for fts in family(ts, state.total_nthreads) or () + # ) + # < slots # Leave room for other family members that will need to run here. + ): + qts = state.queued.peek() if state.validate: assert qts.state == "queued" - assert qts.queued_on is ws, (qts.queued_on, ws) assert qts.key not in recommendations, recommendations[qts.key] + # Task should not have a worker it's intended to run on yet. This is + # guaranteed by the fact that `decide_worker` only puts tasks into `queued` + # if `family_worker(ts) is None`. + # FIXME a scale-up changes the max family size threshold and could + # theoretically turn a previous non-family into a family. + assert ( + fws := family_worker(qts, state.total_nthreads) + ) is None, f"Family of queued task {qts} already has a worker {fws}" + + # Schedule the entire family. + # Note that like `decide_worker`, we allow the family to oversaturate the + # worker, since all tasks in the family must be in memory at once anyway to run + # the downstream task. + # NOTE: we don't ever need to schedule more than one family at once here. + # Since this is called each time 1 task completes, multiple tasks must complete + # for multiple slots to open up. recommendations[qts.key] = "processing" + for fts in family(qts, state.total_nthreads) or (): + if state.validate: + # TODO: remove, would be invalid transition anyway + assert fts.state != "released", fts + + if fts.state == "queued": + recommendations[fts.key] = "processing" + # TODO might be nice to recommend the worker... return ws.address @@ -7747,17 +7721,16 @@ def validate_task_state(ts: TaskState) -> None: assert dts.state != "forgotten" assert (ts.processing_on is not None) == (ts.state == "processing") - assert not (ts.processing_on and ts.queued_on), (ts.processing_on, ts.queued_on) assert bool(ts.who_has) == (ts.state == "memory"), (ts, ts.who_has, ts.state) - if ts.queued_on: - assert ts.state == "queued" - assert ts in ts.queued_on.queued - if ts.state == "queued": - assert ts.queued_on assert not ts.processing_on assert not ts.who_has + assert all(dts.who_has for dts in ts.dependencies), ( + "task queued without all deps", + str(ts), + str(ts.dependencies), + ) if ts.state == "processing": assert all(dts.who_has for dts in ts.dependencies), ( @@ -7766,7 +7739,6 @@ def validate_task_state(ts: TaskState) -> None: str(ts.dependencies), ) assert not ts.waiting_on - assert not ts.queued_on if ts.who_has: assert ts.waiters or ts.who_wants, ( @@ -7800,6 +7772,7 @@ def validate_task_state(ts: TaskState) -> None: if ts.state == "processing": assert ts.processing_on assert ts in ts.processing_on.actors + assert ts.state != "queued" def validate_worker_state(ws: WorkerState) -> None: @@ -7867,6 +7840,54 @@ def worker_saturated(ws: WorkerState, oversaturation_factor: float) -> bool: return task_slots_available(ws, oversaturation_factor) <= 0 +def family_size(ts: TaskState, maxsize: int) -> int | None: + if len(ts.dependents) > maxsize: + return None + size = 0 + for dts in ts.dependents: + # FIXME don't traverse arbitrarily long linear chains!! + while len(dts.dependents) == 1: + dts = next(iter(dts.dependents)) + size += len(dts.dependents) + if size > maxsize: + return None + return size + + +def family(ts: TaskState, maxsize: int) -> set[TaskState] | None: + # if family_size(ts, maxsize) is None: + # return None + # return {fts for dts in ts.dependents for fts in dts.dependencies} + if len(ts.dependents) > maxsize: + return None + tasks: set[TaskState] = set() + for dts in ts.dependents: + # FIXME don't traverse arbitrarily long linear chains!! + while len(dts.dependents) == 1 and len(dts.dependencies) <= 1: + dts = next(iter(dts.dependents)) + if len(tasks) + len(dts.dependencies) > maxsize: + return None + tasks.update(dts.dependencies) + return tasks + + +def family_worker(ts: TaskState, maxsize: int) -> WorkerState | None: + if family_size(ts, maxsize) is None: + return None + # counts = Counter( + # fts.processing_on + # for dts in ts.dependents + # for fts in dts.dependencies + # if fts.processing_on + # ).most_common(1) + counts = Counter( + fts.processing_on for fts in family(ts, maxsize) or () if fts.processing_on + ).most_common(1) + if not counts: + return None + return counts[0][0] + + class KilledWorker(Exception): def __init__(self, task: str, last_worker: WorkerState): super().__init__(task, last_worker) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index cc16309678a..8a78cadb30f 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -147,6 +147,10 @@ def test_decide_worker_coschedule_order_neighbors(ndeps, nthreads): client=True, nthreads=nthreads, config={"distributed.scheduler.work-stealing": False}, + scheduler_kwargs=dict( # TODO remove + dashboard=True, + dashboard_address=":8787", + ), ) async def test_decide_worker_coschedule_order_neighbors_(c, s, *workers): r""" @@ -263,11 +267,12 @@ def random(**kwargs): "distributed.scheduler.work-stealing": False, }, ) -async def test_root_task_overproduction(c, s, *nannies): +async def test_root_task_overproduction(c, s, *nannies: Nanny): """ Workload that would run out of memory and kill workers if >2 root tasks were ever in memory at once on a worker. """ + pids = [n.pid for n in nannies] @delayed(pure=True) def big_data(size: int) -> str: @@ -283,6 +288,9 @@ def big_data(size: int) -> str: await c.compute(final) + # No restarts + assert pids == [n.pid for n in nannies] + @pytest.mark.parametrize( "oversaturation, expected_task_counts", diff --git a/distributed/widgets/templates/worker_state.html.j2 b/distributed/widgets/templates/worker_state.html.j2 index 08629998e1f..cd152080bfc 100644 --- a/distributed/widgets/templates/worker_state.html.j2 +++ b/distributed/widgets/templates/worker_state.html.j2 @@ -3,4 +3,3 @@ status: {{ status }} memory: {{ has_what | length }} processing: {{ processing | length }} - queued: {{ queued | length }} From 0f6603c2578a672734c20c6fbb2d68c2fed7a123 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 11:29:34 -0600 Subject: [PATCH 18/98] Withhold root tasks [no co-assignment] --- distributed/collections.py | 36 ++----- distributed/scheduler.py | 161 ++++------------------------ distributed/tests/test_scheduler.py | 61 +++++++++-- 3 files changed, 81 insertions(+), 177 deletions(-) diff --git a/distributed/collections.py b/distributed/collections.py index b79e0cd6c08..7a9d4300693 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -97,40 +97,18 @@ def pop(self) -> T: self._data.discard(value) return value - def popright(self) -> T: - "Remove and return one of the largest elements (not necessarily the largest)!" - if not self._data: - raise KeyError("popright from an empty set") - while True: - _, _, vref = self._heap.pop() - value = vref() - if value is not None and value in self._data: - self._data.discard(value) - return value - def topk(self, k: int) -> Iterator[T]: - # TODO confirm big-O values here - "Iterator over the largest K elements. This is O(k*logn) for k < n // 2, O(n*logn) otherwise." + "Iterator over the largest K elements. This is O(1) for k == 1, O(n*logn) otherwise." k = min(k, len(self)) if k == 1: yield self.peek() - elif k >= len(self) // 2: - return itertools.islice(self.sorted(), k) else: - # FIXME though neat, with all the list mutation this is probably always slower than sorting inplace. - elems: list[tuple[Any, int, weakref.ref[T]]] = [] - try: - while len(elems) < k: - elem = heapq.heappop(self._heap) - value = elem[-1]() - if value is not None and value in self._data: - # NOTE: we're in a broken state during iteration, since the value exists - # in the set but not the heap. As with all Python iterators, mutating - # while iterating is undefined. - elems.append(elem) - yield value - finally: - self._heap = elems + self._heap + # NOTE: we could pop K items off the queue, then push them back. + # But copying the list K times is probably slower than just sorting it + # with fast C code. + # If we had a `heappop` that sliced the list instead of popping from it, + # we could implement an optimized version for small `k`s. + return itertools.islice(self.sorted(), k) def __iter__(self) -> Iterator[T]: """Iterate over all elements. This is a O(n) operation which returns the diff --git a/distributed/scheduler.py b/distributed/scheduler.py index eb220bb9d3c..cb566509b55 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -16,7 +16,7 @@ import uuid import warnings import weakref -from collections import Counter, defaultdict, deque +from collections import defaultdict, deque from collections.abc import ( Callable, Collection, @@ -1789,31 +1789,22 @@ def decide_worker( # Group is larger than cluster with few dependencies? # Minimize future data transfers. - # TODO update metric to involve families in some way? Most likely not. if ( valid_workers is None and len(tg) > self.total_nthreads * 2 and len(tg.dependencies) < 5 and sum(map(len, tg.dependencies)) < 5 ): - # TODO what max family size cutoff to use? - if ws := family_worker(ts, self.total_nthreads): - # Use the worker where the majority of other tasks are already assigned. - # Even if that worker is already saturated, we let more tasks run on it, - # because all those tasks must be in memory at once anyway to run the - # downstream task. - return ws - - # First task in this family, or such a huge family it doesn't matter where it runs? - # Pick the least busy worker. if not self.idle: - # If all workers are busy, this family gets queued for later. + # All workers busy? Task gets/stays queued. if self.validate: assert ts.key not in recommendations, (ts, recommendations[ts.key]) if ts.state != "queued": recommendations[ts.key] = "queued" return None + # For root tasks, just pick the least busy worker. + # NOTE: this will lead to worst-case scheduling with regards to co-assignment. ws = min( self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads ) @@ -2040,16 +2031,6 @@ def transition_processing_memory( if self.validate: assert not ts.processing_on assert not ts.waiting_on - # processing_recs = { - # k: r for k, r in recommendations.items() if r == "processing" - # } - # assert list(processing_recs) == ( - # sr := sorted( - # processing_recs, - # key=lambda k: self.tasks[k].priority, - # reverse=True, - # ) - # ), (list(processing_recs), sr) return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2441,11 +2422,6 @@ def transition_waiting_queued(self, key, stimulus_id): if self.validate: assert ts not in self.queued - # Task should have gone straight to processing if its family already had a worker - assert family_worker(ts, self.total_nthreads) is None, ( - key, - family(ts, self.total_nthreads), - ) assert not self.idle, (ts, self.idle) # Copied from `transition_waiting_processing` assert not ts.waiting_on @@ -2733,6 +2709,7 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): idle = self.idle saturated = self.saturated + # TODO different metric when `self.WORKER_OVERSATURATION` is `inf`? if not worker_saturated(ws, self.WORKER_OVERSATURATION): idle[ws.address] = ws saturated.discard(ws) @@ -2936,30 +2913,21 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): ts: TaskState recommendations: dict[str, str] = {} + # Schedule any queued tasks onto the new worker if not math.isinf(self.WORKER_OVERSATURATION) and self.queued: - for qts in self.queued.topk( - slots := task_slots_available(ws, self.WORKER_OVERSATURATION) + for qts in reversed( + list( + self.queued.topk( + task_slots_available(ws, self.WORKER_OVERSATURATION) + ) + ) ): if self.validate: assert qts.state == "queued" + assert not qts.processing_on + assert not qts.waiting_on - # Schedule the entire family - for fts in family(qts, self.total_nthreads) or (qts,): - # NOTE: `family` is not priority-ordered, but this is superfluous - # because all tasks in the family will get scheduled anyway - # regardless of order. - if self.validate: - # If any tasks in a family are running or have run, no other tasks - # in that family should have been queued---they'd go straight to `processing`. - assert fts.state in ("waiting", "queued"), ( - fts, - family(qts, self.total_nthreads), - ) - slots -= 1 # We allocate slots for waiting tasks as well - if fts.state == "queued": - recommendations[fts.key] = "processing" - if slots <= 0: - break + recommendations[qts.key] = "processing" now_runnable: list[TaskState] = [] for ts in self.unrunnable: @@ -4600,10 +4568,6 @@ def validate_queued(self, key): for dts in ts.dependencies: assert dts.who_has assert ts in dts.waiters - assert family_worker(ts, self.total_nthreads) is None, ( - key, - family(ts, self.total_nthreads), - ) def validate_processing(self, key): ts: TaskState = self.tasks[key] @@ -7287,7 +7251,7 @@ def adaptive_target(self, target_duration=None): ) # TODO: threads per worker # Avoid a few long tasks from asking for many cores - tasks_processing = 0 + tasks_processing = len(self.queued) for ws in self.workers.values(): tasks_processing += len(ws.processing) @@ -7399,50 +7363,17 @@ def _remove_from_processing( state.check_idle_saturated(ws) state.release_resources(ts, ws) - # If a slot has opened up for a queued task, schedule it and its whole family. - if ( - state.queued - and task_slots_available(ws, state.WORKER_OVERSATURATION) > 0 - # TODO: leaving room causes deadlock when `ts` is a downstream, non-rootish task - # that just happens to meet the family criteria (and some of the family is waiting - # on queued tasks). - # But without this, we could have overproduction when some family members have deps - # and others don't (since we'll let a new family come in and start on the worker - # before the current family is done). - # and sum( - # fts.state == "waiting" for fts in family(ts, state.total_nthreads) or () - # ) - # < slots # Leave room for other family members that will need to run here. - ): + # If a slot has opened up for a queued task, schedule it. + if state.queued and not worker_saturated(ws, state.WORKER_OVERSATURATION): qts = state.queued.peek() if state.validate: assert qts.state == "queued" assert qts.key not in recommendations, recommendations[qts.key] - # Task should not have a worker it's intended to run on yet. This is - # guaranteed by the fact that `decide_worker` only puts tasks into `queued` - # if `family_worker(ts) is None`. - # FIXME a scale-up changes the max family size threshold and could - # theoretically turn a previous non-family into a family. - assert ( - fws := family_worker(qts, state.total_nthreads) - ) is None, f"Family of queued task {qts} already has a worker {fws}" - - # Schedule the entire family. - # Note that like `decide_worker`, we allow the family to oversaturate the - # worker, since all tasks in the family must be in memory at once anyway to run - # the downstream task. - # NOTE: we don't ever need to schedule more than one family at once here. - # Since this is called each time 1 task completes, multiple tasks must complete - # for multiple slots to open up. - recommendations[qts.key] = "processing" - for fts in family(qts, state.total_nthreads) or (): - if state.validate: - # TODO: remove, would be invalid transition anyway - assert fts.state != "released", fts - if fts.state == "queued": - recommendations[fts.key] = "processing" - # TODO might be nice to recommend the worker... + # NOTE: we don't need to schedule more than one task at once here. Since this is + # called each time 1 task completes, multiple tasks must complete for multiple + # slots to open up. + recommendations[qts.key] = "processing" return ws.address @@ -7840,54 +7771,6 @@ def worker_saturated(ws: WorkerState, oversaturation_factor: float) -> bool: return task_slots_available(ws, oversaturation_factor) <= 0 -def family_size(ts: TaskState, maxsize: int) -> int | None: - if len(ts.dependents) > maxsize: - return None - size = 0 - for dts in ts.dependents: - # FIXME don't traverse arbitrarily long linear chains!! - while len(dts.dependents) == 1: - dts = next(iter(dts.dependents)) - size += len(dts.dependents) - if size > maxsize: - return None - return size - - -def family(ts: TaskState, maxsize: int) -> set[TaskState] | None: - # if family_size(ts, maxsize) is None: - # return None - # return {fts for dts in ts.dependents for fts in dts.dependencies} - if len(ts.dependents) > maxsize: - return None - tasks: set[TaskState] = set() - for dts in ts.dependents: - # FIXME don't traverse arbitrarily long linear chains!! - while len(dts.dependents) == 1 and len(dts.dependencies) <= 1: - dts = next(iter(dts.dependents)) - if len(tasks) + len(dts.dependencies) > maxsize: - return None - tasks.update(dts.dependencies) - return tasks - - -def family_worker(ts: TaskState, maxsize: int) -> WorkerState | None: - if family_size(ts, maxsize) is None: - return None - # counts = Counter( - # fts.processing_on - # for dts in ts.dependents - # for fts in dts.dependencies - # if fts.processing_on - # ).most_common(1) - counts = Counter( - fts.processing_on for fts in family(ts, maxsize) or () if fts.processing_on - ).most_common(1) - if not counts: - return None - return counts[0][0] - - class KilledWorker(Exception): def __init__(self, task: str, last_worker: WorkerState): super().__init__(task, last_worker) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 8a78cadb30f..5e0d7a07393 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -134,6 +134,7 @@ async def test_decide_worker_with_restrictions(client, s, a, b, c): assert x.key in a.data or x.key in b.data +@pytest.mark.skip("Current queuing does not support co-assignment") @pytest.mark.parametrize("ndeps", [0, 1, 4]) @pytest.mark.parametrize( "nthreads", @@ -255,7 +256,6 @@ def random(**kwargs): client=True, nthreads=[("", 2)] * 2, worker_kwargs={"memory_limit": "1.0GiB"}, - timeout=3600, # TODO remove Worker=Nanny, scheduler_kwargs=dict( # TODO remove dashboard=True, @@ -328,6 +328,7 @@ async def _test_oversaturation_factor(c, s, a, b): _test_oversaturation_factor() +@pytest.mark.skip("Current queuing does not support co-assignment") @pytest.mark.parametrize( "saturation_factor", [ @@ -357,19 +358,55 @@ async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor assert len(b.tasks) == 9 +@pytest.mark.slow @gen_cluster( client=True, nthreads=[("", 2)] * 2, - timeout=3600, # TODO remove scheduler_kwargs=dict( # TODO remove dashboard=True, dashboard_address=":8787", ), ) -async def test_queued_tasks_rebalance(c, s, a, b): - event = Event() - fs = c.map(lambda _: event.wait(), range(100)) - await c.gather(fs) +async def test_queued_tasks_rebalance(client, s, a, b): + """ + Test that all queued tasks complete as workers come and go. + + Does not test how well balanced the load was. + """ + roots1 = [delayed(slowinc)(i) for i in range(400)] + roots2 = [delayed(slowinc)(i, delay=0.01) for i in range(len(roots1))] + combined = [x + y for x, y in zip(roots1, roots2)] + agg = [sum(xs) for xs in partition(4, combined)] + + fs = client.compute(agg) + + while len(a.data) + len(b.data) < len(agg) * 0.25: + await asyncio.sleep(0.01) + + # Add a new worker + async with Worker(s.address, nthreads=2) as c: + while not c.data: + await asyncio.sleep(0.01) + + # Now add another + async with Worker(s.address, nthreads=1) as d: + while not d.data: + await asyncio.sleep(0.01) + + # Remove an existing worker + await a.close() + + while len(b.data) + len(c.data) + len(d.data) < len(agg) * 0.75: + await asyncio.sleep(0.01) + + # And a new one + await d.close() + + await client.gather(fs) + assert a.tasks + assert b.tasks + assert c.tasks + assert d.tasks @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) @@ -448,7 +485,7 @@ async def test_remove_worker_from_scheduler(s, a, b): assert a.address in s.stream_comms await s.remove_worker(address=a.address, stimulus_id="test") assert a.address not in s.workers - assert len(s.workers[b.address].processing) == len(dsk) # b owns everything + assert len(s.workers[b.address].processing) + len(s.queued) == len(dsk) @gen_cluster() @@ -730,12 +767,18 @@ async def test_ready_remove_worker(s, a, b): dependencies={"x-%d" % i: [] for i in range(20)}, ) - assert all(len(w.processing) > w.nthreads for w in s.workers.values()) + assert all(len(w.processing) == w.nthreads for w in s.workers.values()) + assert sum(len(w.processing) for w in s.workers.values()) + len(s.queued) == len( + s.tasks + ) await s.remove_worker(address=a.address, stimulus_id="test") assert set(s.workers) == {b.address} - assert all(len(w.processing) > w.nthreads for w in s.workers.values()) + assert all(len(w.processing) == w.nthreads for w in s.workers.values()) + assert sum(len(w.processing) for w in s.workers.values()) + len(s.queued) == len( + s.tasks + ) @gen_cluster(client=True, Worker=Nanny, timeout=60) From e10fdca6f1941606b40cf866bb588d5c805b99ab Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 12:00:37 -0600 Subject: [PATCH 19/98] Factor out `_add_to_processing` --- distributed/scheduler.py | 87 +++++++++++++++------------------------- 1 file changed, 32 insertions(+), 55 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index cb566509b55..a88597f5ad7 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1852,38 +1852,11 @@ def decide_worker( 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, recommendations) - 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)] + _add_to_processing(self, ts, recommendations, worker_msgs) return recommendations, client_msgs, worker_msgs except Exception as e: logger.exception(e) @@ -2492,35 +2465,9 @@ def transition_queued_processing(self, key, stimulus_id): if self.validate: assert not ts.actor, "Actors can't be queued wat" assert ts in self.queued - # Copied from `transition_waiting_processing` - assert not ts.processing_on - assert not ts.waiting_on - assert not ts.who_has - assert not ts.exception_blame - assert not ts.has_lost_dependencies - assert ts not in self.unrunnable - assert all(dts.who_has for dts in ts.dependencies) - # NOTE: if all workers are now saturated and this task shouldn't actually run, `ws` is be None. - if ws := self.decide_worker(ts, recommendations): - self.queued.remove(ts) - # TODO Copied from `transition_waiting_processing`; factor out into helper function - 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 + _add_to_processing(self, ts, recommendations, worker_msgs) - if ts.actor: - ws.actors.add(ts) - - # logger.debug("Send job to worker: %s, %s", worker, key) - - worker_msgs[ws.address] = [_task_to_msg(self, ts)] - - if self.validate: - assert not recommendations, recommendations return recommendations, client_msgs, worker_msgs except Exception as e: logger.exception(e) @@ -7335,6 +7282,36 @@ def request_remove_replicas( ) +def _add_to_processing( + state: SchedulerState, ts: TaskState, recommendations: dict, worker_msgs: dict +) -> WorkerState | None: + if state.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 state.unrunnable + assert all(dts.who_has for dts in ts.dependencies) + + if ws := state.decide_worker(ts, recommendations): + state._set_duration_estimate(ts, ws) + ts.processing_on = ws + ts.state = "processing" + state.queued.discard(ts) + state.consume_resources(ts, ws) + state.check_idle_saturated(ws) + state.n_tasks += 1 + + if ts.actor: + ws.actors.add(ts) + + # logger.debug("Send job to worker: %s, %s", worker, key) + + worker_msgs[ws.address] = [_task_to_msg(state, ts)] + return ws + + def _remove_from_processing( state: SchedulerState, ts: TaskState, recommendations: dict ) -> str | None: From 3eb1d6801b3ee2570c2b42971591d317b1b62710 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 12:05:03 -0600 Subject: [PATCH 20/98] Factor out `_propagage_released` --- distributed/scheduler.py | 62 ++++++++++++++++++---------------------- 1 file changed, 28 insertions(+), 34 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index a88597f5ad7..8ccb77bc980 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2238,24 +2238,7 @@ def transition_processing_released(self, key, 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 - + _propagage_released(self, ts, recommendations) return recommendations, client_msgs, worker_msgs except Exception as e: logger.exception(e) @@ -2430,22 +2413,7 @@ def transition_queued_released(self, key, stimulus_id): self.queued.remove(ts) - # TODO copied from `transition_processing_released`; factor out into helper function - 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() - + _propagage_released(self, ts, recommendations) return recommendations, client_msgs, worker_msgs except Exception as e: logger.exception(e) @@ -7413,6 +7381,32 @@ def _add_to_memory( ) +def _propagage_released( + state: SchedulerState, + ts: TaskState, + recommendations: dict[str, str], +) -> None: + ts.state = "released" + key = ts.key + + 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 state.validate: + assert not ts.processing_on + assert ts not in state.queued + + def _propagate_forgotten( state: SchedulerState, ts: TaskState, From c685b3c922563a161778479b2ad107b7f9515da6 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 12:31:29 -0600 Subject: [PATCH 21/98] Update `check_idle_saturated` Update docstring and add back logic for queuing disabled case --- distributed/scheduler.py | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 8ccb77bc980..ebf163897d5 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1795,6 +1795,12 @@ def decide_worker( and len(tg.dependencies) < 5 and sum(map(len, tg.dependencies)) < 5 ): + if math.isinf(self.WORKER_SATURATION): + return min( + (self.idle or self.workers).values(), + key=lambda ws: len(ws.processing) / ws.nthreads, + ) + if not self.idle: # All workers busy? Task gets/stays queued. if self.validate: @@ -2609,7 +2615,16 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): 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. TODO update + large enough. + + If ``distributed.scheduler.worker-oversaturation`` is not ``inf`` + (scheduler-side queuing is enabled), they are considered idle + if they have fewer tasks processing than the ``worker-oversaturation`` + threshold dictates. + + Otherwise, they are considered idle if they have fewer tasks processing + than threads, or if their tasks' total expected runtime is less than half + the expected runtime of the same number of average tasks. This is useful for load balancing and adaptivity. """ @@ -2624,14 +2639,16 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): idle = self.idle saturated = self.saturated - # TODO different metric when `self.WORKER_OVERSATURATION` is `inf`? - if not worker_saturated(ws, self.WORKER_OVERSATURATION): + if ( + (p < nc or occ < nc * avg / 2) + if math.isinf(self.WORKER_OVERSATURATION) + else not worker_saturated(ws, self.WORKER_OVERSATURATION) + ): idle[ws.address] = ws saturated.discard(ws) else: idle.pop(ws.address, None) - # TODO do we still want any of this? if p > nc: pending: float = occ * (p - nc) / (p * nc) if 0.4 < pending > 1.9 * avg: From e1dda98e043c22d738d9b8f9d136144ea54f588d Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 12:35:53 -0600 Subject: [PATCH 22/98] Fix `topk` for 0/negative values --- distributed/collections.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/distributed/collections.py b/distributed/collections.py index 7a9d4300693..f3ecae68cd3 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -99,7 +99,8 @@ def pop(self) -> T: def topk(self, k: int) -> Iterator[T]: "Iterator over the largest K elements. This is O(1) for k == 1, O(n*logn) otherwise." - k = min(k, len(self)) + if k <= 0: + return # empty iterator if k == 1: yield self.peek() else: From f811246f677dff2e3cba8d4803908aaed079a9a5 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 12:51:32 -0600 Subject: [PATCH 23/98] Tests for HeapSet.topk --- distributed/collections.py | 2 +- distributed/tests/test_collections.py | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/distributed/collections.py b/distributed/collections.py index f3ecae68cd3..1462888050e 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -109,7 +109,7 @@ def topk(self, k: int) -> Iterator[T]: # with fast C code. # If we had a `heappop` that sliced the list instead of popping from it, # we could implement an optimized version for small `k`s. - return itertools.islice(self.sorted(), k) + yield from itertools.islice(self.sorted(), k) def __iter__(self) -> Iterator[T]: """Iterate over all elements. This is a O(n) operation which returns the diff --git a/distributed/tests/test_collections.py b/distributed/tests/test_collections.py index 25853495ebf..2ab5ce1f2e8 100644 --- a/distributed/tests/test_collections.py +++ b/distributed/tests/test_collections.py @@ -32,6 +32,9 @@ def __hash__(self): def __eq__(self, other): return isinstance(other, C) and other.k == self.k + def __repr__(self): + return f"C({self.k}, {self.i})" + heap = HeapSet(key=lambda c: c.i) cx = C("x", 2) @@ -123,6 +126,21 @@ def __eq__(self, other): heap.add(cx) assert cx in heap + # Test topk() + heap.add(cy) + heap.add(cw) + heap.add(cz) + heap.add(cx) + assert list(heap.topk(3)) == [cy, cx, cz] + heap.remove(cz) + assert list(heap.topk(10)) == [cy, cx, cw] + assert list(heap.topk(0)) == [] + assert list(heap.topk(-1)) == [] + heap.remove(cy) + assert list(heap.topk(1)) == [cx] + heap.remove(cw) + assert list(heap.topk(1)) == [cx] + # Test resilience to failure in key() bad_key = C("bad_key", 0) del bad_key.i From d347b3258328ce07204f1021744fb63b492f59b9 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 17:04:07 -0600 Subject: [PATCH 24/98] fix mypy --- distributed/tests/test_scheduler.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 5e0d7a07393..d9710a1d8da 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -267,14 +267,14 @@ def random(**kwargs): "distributed.scheduler.work-stealing": False, }, ) -async def test_root_task_overproduction(c, s, *nannies: Nanny): +async def test_root_task_overproduction(c, s, *nannies): """ Workload that would run out of memory and kill workers if >2 root tasks were ever in memory at once on a worker. """ pids = [n.pid for n in nannies] - @delayed(pure=True) + @delayed(pure=True) # type: ignore def big_data(size: int) -> str: return "x" * size @@ -303,7 +303,9 @@ def big_data(size: int) -> str: # ^ depends on root task assignment logic; ok if changes, just needs to add up to 10 ], ) -def test_oversaturation_factor(oversaturation, expected_task_counts: tuple[int, int]): +def test_oversaturation_factor( + oversaturation: int | float, expected_task_counts: tuple[int, int] +) -> None: @gen_cluster( client=True, nthreads=[("", 2), ("", 1)], From 1990dd7751277fc6aa380ebdb1dd1280fc2d116f Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 13:44:56 -0600 Subject: [PATCH 25/98] worker-oversaturation -> worker-saturation Just easier to explain this way --- distributed/distributed-schema.yaml | 18 ++++++------- distributed/distributed.yaml | 2 +- distributed/scheduler.py | 41 +++++++++++++---------------- distributed/tests/test_scheduler.py | 30 +++++++++++---------- 4 files changed, 45 insertions(+), 46 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index aa42c08972a..337f2fd1b0e 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -117,24 +117,24 @@ properties: description: | How frequently to balance worker loads - worker-oversaturation: + worker-saturation: type: float description: | - Controls how many extra root tasks are sent to workers (like a `readahead`). + Controls how many root tasks are sent to workers (like a `readahead`). - `floor(worker-oversaturation * worker.nthreads)` _extra_ tasks are sent to the worker - beyond its thread count. If `.inf`, all runnable tasks are immediately sent to workers. + `max(floor(worker-oversaturation * worker.nthreads), 1)` root tasks are sent to a + worker at a time. If `.inf`, all runnable tasks are immediately sent to workers. - Allowing oversaturation means a worker will start running a new root task as soon as - it completes the previous, even if there is a higher-priority downstream task to run. - This reduces worker idleness, by letting workers do something while waiting for further - instructions from the scheduler. + Allowing oversaturation (> 1.0) means a worker will start running a new root task as + soon as it completes the previous, even if there is a higher-priority downstream task + to run. This reduces worker idleness, by letting workers do something while waiting for + further instructions from the scheduler. This generally comes at the expense of increased memory usage. It leads to "wider" (more breadth-first) execution of the graph. Compute-bound workloads benefit from oversaturation. Memory-bound workloads should - generally leave `worker-oversaturation` at 0, though 0.25-0.5 could slightly improve + generally leave `worker-saturation` at 1.0, though 1.25-1.5 could slightly improve performance if ample memory is available. worker-ttl: diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index a37e1c34add..124e001b0c4 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -22,7 +22,7 @@ distributed: events-log-length: 100000 work-stealing: True # workers should steal tasks from each other work-stealing-interval: 100ms # Callback time for work stealing - worker-oversaturation: 0.0 # Send this fraction of nthreads extra root tasks to workers + worker-saturation: 1.0 # Send this fraction of nthreads root tasks to workers worker-ttl: "5 minutes" # like '60s'. Time to live for workers. They must heartbeat faster than this pickle: True # Is the scheduler allowed to deserialize arbitrary bytestrings preload: [] # Run custom modules with Scheduler diff --git a/distributed/scheduler.py b/distributed/scheduler.py index ebf163897d5..fa7d55b987a 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1282,7 +1282,7 @@ class SchedulerState: "MEMORY_REBALANCE_SENDER_MIN", "MEMORY_REBALANCE_RECIPIENT_MAX", "MEMORY_REBALANCE_HALF_GAP", - "WORKER_OVERSATURATION", + "WORKER_SATURATION", } def __init__( @@ -1353,8 +1353,8 @@ def __init__( dask.config.get("distributed.worker.memory.rebalance.sender-recipient-gap") / 2.0 ) - self.WORKER_OVERSATURATION = dask.config.get( - "distributed.scheduler.worker-oversaturation" + self.WORKER_SATURATION = dask.config.get( + "distributed.scheduler.worker-saturation" ) self.transition_counter = 0 self._idle_transition_counter = 0 @@ -1815,9 +1815,9 @@ def decide_worker( self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads ) if self.validate: - assert not worker_saturated(ws, self.WORKER_OVERSATURATION), ( + assert not worker_saturated(ws, self.WORKER_SATURATION), ( ws, - task_slots_available(ws, self.WORKER_OVERSATURATION), + task_slots_available(ws, self.WORKER_SATURATION), ) return ws @@ -2617,9 +2617,9 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): all of their threads, and if the expected runtime of those tasks is large enough. - If ``distributed.scheduler.worker-oversaturation`` is not ``inf`` + If ``distributed.scheduler.worker-saturation`` is not ``inf`` (scheduler-side queuing is enabled), they are considered idle - if they have fewer tasks processing than the ``worker-oversaturation`` + if they have fewer tasks processing than the ``worker-saturation`` threshold dictates. Otherwise, they are considered idle if they have fewer tasks processing @@ -2641,8 +2641,8 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): saturated = self.saturated if ( (p < nc or occ < nc * avg / 2) - if math.isinf(self.WORKER_OVERSATURATION) - else not worker_saturated(ws, self.WORKER_OVERSATURATION) + if math.isinf(self.WORKER_SATURATION) + else not worker_saturated(ws, self.WORKER_SATURATION) ): idle[ws.address] = ws saturated.discard(ws) @@ -2846,13 +2846,9 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): recommendations: dict[str, str] = {} # Schedule any queued tasks onto the new worker - if not math.isinf(self.WORKER_OVERSATURATION) and self.queued: + if not math.isinf(self.WORKER_SATURATION) and self.queued: for qts in reversed( - list( - self.queued.topk( - task_slots_available(ws, self.WORKER_OVERSATURATION) - ) - ) + list(self.queued.topk(task_slots_available(ws, self.WORKER_SATURATION))) ): if self.validate: assert qts.state == "queued" @@ -7277,6 +7273,7 @@ def _add_to_processing( assert not ts.processing_on assert not ts.has_lost_dependencies assert ts not in state.unrunnable + assert ts not in state.queued assert all(dts.who_has for dts in ts.dependencies) if ws := state.decide_worker(ts, recommendations): @@ -7326,7 +7323,7 @@ def _remove_from_processing( state.release_resources(ts, ws) # If a slot has opened up for a queued task, schedule it. - if state.queued and not worker_saturated(ws, state.WORKER_OVERSATURATION): + if state.queued and not worker_saturated(ws, state.WORKER_SATURATION): qts = state.queued.peek() if state.validate: assert qts.state == "queued" @@ -7746,17 +7743,17 @@ def heartbeat_interval(n: int) -> float: return n / 200 + 1 -def task_slots_available(ws: WorkerState, oversaturation_factor: float) -> int: +def task_slots_available(ws: WorkerState, saturation_factor: float) -> int: "Number of tasks that can be sent to this worker without oversaturating it" - assert not math.isinf(oversaturation_factor) + assert not math.isinf(saturation_factor) nthreads = ws.nthreads - return max(nthreads + int(oversaturation_factor * nthreads), 1) - len(ws.processing) + return max(int(saturation_factor * nthreads), 1) - len(ws.processing) -def worker_saturated(ws: WorkerState, oversaturation_factor: float) -> bool: - if math.isinf(oversaturation_factor): +def worker_saturated(ws: WorkerState, saturation_factor: float) -> bool: + if math.isinf(saturation_factor): return False - return task_slots_available(ws, oversaturation_factor) <= 0 + return task_slots_available(ws, saturation_factor) <= 0 class KilledWorker(Exception): diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index d9710a1d8da..2fa8228233e 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -293,32 +293,34 @@ def big_data(size: int) -> str: @pytest.mark.parametrize( - "oversaturation, expected_task_counts", + "saturation, expected_task_counts", [ - (1.5, (5, 2)), - (1.0, (4, 2)), - (0.0, (2, 1)), + (2.5, (5, 2)), + (2.0, (4, 2)), + (1.0, (2, 1)), (-1.0, (1, 1)), (float("inf"), (7, 3)) # ^ depends on root task assignment logic; ok if changes, just needs to add up to 10 ], ) -def test_oversaturation_factor( - oversaturation: int | float, expected_task_counts: tuple[int, int] +def test_saturation_factor( + saturation: int | float, expected_task_counts: tuple[int, int] ) -> None: @gen_cluster( client=True, nthreads=[("", 2), ("", 1)], config={ - "distributed.scheduler.worker-oversaturation": oversaturation, + "distributed.scheduler.worker-saturation": saturation, }, ) - async def _test_oversaturation_factor(c, s, a, b): + async def _test_saturation_factor(c, s, a, b): event = Event() - fs = c.map(lambda _: event.wait(), range(10)) + fs = c.map( + lambda _: event.wait(), range(10), key=[f"wait-{i}" for i in range(10)] + ) while a.state.executing_count < min( - a.nthreads, expected_task_counts[0] - ) or b.state.executing_count < min(b.nthreads, expected_task_counts[1]): + a.state.nthreads, expected_task_counts[0] + ) or b.state.executing_count < min(b.state.nthreads, expected_task_counts[1]): await asyncio.sleep(0.01) assert len(a.state.tasks) == expected_task_counts[0] @@ -327,15 +329,15 @@ async def _test_oversaturation_factor(c, s, a, b): await event.set() await c.gather(fs) - _test_oversaturation_factor() + _test_saturation_factor() @pytest.mark.skip("Current queuing does not support co-assignment") @pytest.mark.parametrize( "saturation_factor", [ - 0.0, 1.0, + 2.0, pytest.param( float("inf"), marks=pytest.mark.skip("https://github.com/dask/distributed/issues/6597"), @@ -347,7 +349,7 @@ async def _test_oversaturation_factor(c, s, a, b): nthreads=[("", 2), ("", 1)], ) async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor): - s.WORKER_OVERSATURATION = saturation_factor + s.WORKER_SATURATION = saturation_factor xs = [delayed(i, name=f"x-{i}") for i in range(9)] ys = [delayed(i, name=f"y-{i}") for i in range(9)] zs = [x + y for x, y in zip(xs, ys)] From be1b9cad99eb514565fdbfe07a0203ede276285a Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 16:36:34 -0600 Subject: [PATCH 26/98] fixup! Factor out `_add_to_processing` --- distributed/scheduler.py | 1 - 1 file changed, 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index fa7d55b987a..08eea409266 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7273,7 +7273,6 @@ def _add_to_processing( assert not ts.processing_on assert not ts.has_lost_dependencies assert ts not in state.unrunnable - assert ts not in state.queued assert all(dts.who_has for dts in ts.dependencies) if ws := state.decide_worker(ts, recommendations): From 85f912037db23d1aad599c57814c6d67cbe2a4ca Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 16:42:52 -0600 Subject: [PATCH 27/98] fix `test_queued_tasks_rebalance` --- distributed/tests/test_scheduler.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 2fa8228233e..b4ee9ad5a59 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -407,10 +407,10 @@ async def test_queued_tasks_rebalance(client, s, a, b): await d.close() await client.gather(fs) - assert a.tasks - assert b.tasks - assert c.tasks - assert d.tasks + assert a.state.tasks + assert b.state.tasks + assert c.state.tasks + assert d.state.tasks @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) From bb08c8de7366a154c1625295a25c71514cf951a9 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 22 Jun 2022 16:57:10 -0600 Subject: [PATCH 28/98] Fix occupancy tests I think this fix is reasonable? I wonder if occupancy should include queued tasks though? --- distributed/tests/test_scheduler.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index b4ee9ad5a59..454b0fce2a1 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -1266,9 +1266,13 @@ async def test_learn_occupancy(c, s, a, b): while sum(len(ts.who_has) for ts in s.tasks.values()) < 10: await asyncio.sleep(0.01) - assert 100 < s.total_occupancy < 1000 + nproc = sum(ts.state == "processing" for ts in s.tasks.values()) + assert nproc * 0.1 < s.total_occupancy < nproc * 0.4 for w in [a, b]: - assert 50 < s.workers[w.address].occupancy < 700 + ws = s.workers[w.address] + occ = ws.occupancy + proc = len(ws.processing) + assert proc * 0.1 < occ < proc * 0.4 @pytest.mark.slow @@ -1279,7 +1283,8 @@ async def test_learn_occupancy_2(c, s, a, b): while not any(ts.who_has for ts in s.tasks.values()): await asyncio.sleep(0.01) - assert 100 < s.total_occupancy < 1000 + nproc = sum(ts.state == "processing" for ts in s.tasks.values()) + assert nproc * 0.1 < s.total_occupancy < nproc * 0.4 @gen_cluster(client=True) From 966d61f546ec9888655798f44b78320ea33c0388 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 14:17:10 -0600 Subject: [PATCH 29/98] Test releasing previously queued paused tasks Tasks shouldn't be both `no-worker` and in the queue. If all workers are paused, tasks will currently to go `no-worker`, even if they're queued. If we then try to schedule them (because a slot opens up from task completion, tasks released, new worker joining, etc.) we find an invalid state. --- distributed/scheduler.py | 6 +++- distributed/tests/test_scheduler.py | 45 +++++++++++++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 08eea409266..88c575078d2 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -4493,6 +4493,9 @@ def validate_queued(self, key): assert not ts.waiting_on assert not ts.who_has assert not ts.processing_on + assert not ( + ts.worker_restrictions or ts.host_restrictions or ts.resource_restrictions + ) for dts in ts.dependencies: assert dts.who_has assert ts in dts.waiters @@ -4860,6 +4863,7 @@ def handle_worker_status_change( "status": status, }, ) + logger.debug(f"Worker status {prev_status.name} -> {status} - {ws}") if ws.status == Status.running: self.running.add(ws) @@ -7325,7 +7329,7 @@ def _remove_from_processing( if state.queued and not worker_saturated(ws, state.WORKER_SATURATION): qts = state.queued.peek() if state.validate: - assert qts.state == "queued" + assert qts.state == "queued", qts.state assert qts.key not in recommendations, recommendations[qts.key] # NOTE: we don't need to schedule more than one task at once here. Since this is diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 454b0fce2a1..345a0e48fe3 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -292,6 +292,51 @@ def big_data(size: int) -> str: assert pids == [n.pid for n in nannies] +@pytest.mark.parametrize("withhold", [True, False]) +@gen_cluster( + client=True, + nthreads=[("", 2)] * 2, + worker_kwargs={"memory_limit": "1.0 GiB"}, + Worker=Nanny, + scheduler_kwargs=dict( # TODO remove + dashboard=True, + dashboard_address=":8787", + ), + config={ + # With typical overhead, 1 task can be in memory but the second will trigger a pause + "distributed.worker.memory.pause": 0.4, + "distributed.worker.memory.target": False, + "distributed.worker.memory.spill": False, + "distributed.scheduler.work-stealing": False, + }, +) +async def test_queued_paused_released(c, s, a, b, withhold): + if not withhold: + s.WORKER_SATURATION = float("inf") + + @delayed(pure=True) # type: ignore + def big_data(size: int) -> str: + return "x" * size + + roots = [ + big_data(parse_bytes("200 MiB"), dask_key_name=f"root-{i}") for i in range(16) + ] + memory_consumed = [delayed(len)(x) for x in roots] + f = c.compute(sum(memory_consumed)) + + while s.running: # wait for both workers pausing to hit the scheduler + await asyncio.sleep(0.01) + + async with Worker(s.address, nthreads=2) as w: + # Tasks are successfully scheduled onto a new worker + while not w.state.data: + await asyncio.sleep(0.01) + + f.release() + while s.tasks: + await asyncio.sleep(0.01) + + @pytest.mark.parametrize( "saturation, expected_task_counts", [ From 15494f0d1039f97fde32b2c3bb4ef40c2c0eff24 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 14:18:16 -0600 Subject: [PATCH 30/98] driveby: fix transition debug log end state This was logging the actual end state, instead of the recommended end state --- distributed/scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 88c575078d2..89fb9484df2 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1527,7 +1527,7 @@ def _transition( "Transitioned %r %s->%s (actual: %s). Consequence: %s", key, start, - finish2, + finish, ts.state, dict(recommendations), ) From 546aa4a8500b1c62cdab4ee90b34d28d70167b71 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 14:33:11 -0600 Subject: [PATCH 31/98] Refactor scheduling when no workers are running If all workers were paused, we would put tasks in the `no-worker` state. Now that `queued` is a thing, we want queued tasks in this case to just stay on the queue, and not be added to `unrunnable`. This commit takes the opposite of @crusaderky's view in https://github.com/dask/distributed/pull/5665/files#r787886583, and makes `idle` always a subset of `running`. Even if pedantically, the name `idle` isn't quite accurate, `idle` is typically _used_ as the set of "prime candidate for new tasks", so we make it that way. We do this to maintain the invariant that `valid_workers` always returns None if the task doesn't have restrictions. Our root task detection logic relied on this, as did the `not ts.loose_restrictions` check. Otherwise, when some workers are paused, root tasks will no longer be scheduled in the typical way. There are other approaches here which might be simpler, which I'll explore in following commits. --- distributed/scheduler.py | 70 ++++++++++++++++++++++++++++++++-------- 1 file changed, 56 insertions(+), 14 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 89fb9484df2..0b3dc9d18d2 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1783,8 +1783,7 @@ def decide_worker( and not valid_workers and not ts.loose_restrictions ): - self.unrunnable.add(ts) - ts.state = "no-worker" + recommendations[ts.key] = "no-worker" return None # Group is larger than cluster with few dependencies? @@ -1796,10 +1795,12 @@ def decide_worker( and sum(map(len, tg.dependencies)) < 5 ): if math.isinf(self.WORKER_SATURATION): - return min( - (self.idle or self.workers).values(), - key=lambda ws: len(ws.processing) / ws.nthreads, - ) + pool = self.idle.values() if self.idle else self.running + if not pool: + recommendations[ts.key] = "no-worker" + return None + + return min(pool, key=lambda ws: len(ws.processing) / ws.nthreads) if not self.idle: # All workers busy? Task gets/stays queued. @@ -1819,12 +1820,20 @@ def decide_worker( ws, task_slots_available(ws, self.WORKER_SATURATION), ) + assert ws in self.running, (ws, self.running) return ws + if valid_workers is None and len(self.running) < len(self.workers): + if not self.running: + recommendations[ts.key] = "no-worker" + return None + + valid_workers = self.running + if ts.dependencies or valid_workers is not None: ws = decide_worker( ts, - self.workers.values(), + self.running, valid_workers, partial(self.worker_objective, ts), ) @@ -1852,6 +1861,7 @@ def decide_worker( if self.validate and ws is not None: assert ws.address in self.workers + assert ws in self.running, (ws, self.running) return ws @@ -2406,6 +2416,32 @@ def transition_waiting_queued(self, key, stimulus_id): pdb.set_trace() raise + def transition_waiting_no_worker(self, key, stimulus_id): + try: + ts: TaskState = self.tasks[key] + recommendations: dict = {} + client_msgs: dict = {} + worker_msgs: dict = {} + + if self.validate: + assert ts not in self.queued + assert not ts.who_has + assert not ts.exception_blame + assert not ts.processing_on + assert ts not in self.unrunnable + + ts.state = "no-worker" + self.unrunnable.add(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_queued_released(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] @@ -2556,6 +2592,7 @@ def transition_released_forgotten(self, key, stimulus_id): ("released", "waiting"): transition_released_waiting, ("waiting", "released"): transition_waiting_released, ("waiting", "processing"): transition_waiting_processing, + ("waiting", "no-worker"): transition_waiting_no_worker, ("waiting", "queued"): transition_waiting_queued, ("waiting", "memory"): transition_waiting_memory, ("queued", "released"): transition_queued_released, @@ -2644,7 +2681,8 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): if math.isinf(self.WORKER_SATURATION) else not worker_saturated(ws, self.WORKER_SATURATION) ): - idle[ws.address] = ws + if ws.status == Status.running: + idle[ws.address] = ws saturated.discard(ws) else: idle.pop(ws.address, None) @@ -2691,7 +2729,8 @@ def get_task_duration(self, ts: TaskState) -> float: def valid_workers(self, ts: TaskState) -> set: # set[WorkerState] | None """Return set of currently valid workers for key - If all workers are valid then this returns ``None``. + If all workers are valid then this returns ``None``, in which case + any running worker can be used. This checks tracks the following state: * worker_restrictions @@ -2740,10 +2779,7 @@ def valid_workers(self, ts: TaskState) -> set: # set[WorkerState] | None else: s &= ww - if s is None: - if len(self.running) < len(self.workers): - return self.running.copy() - else: + if s: s = {self.workers[addr] for addr in s} if len(self.running) < len(self.workers): s &= self.running @@ -4575,13 +4611,17 @@ def validate_state(self, allow_overlap: bool = False) -> None: if not (set(self.workers) == set(self.stream_comms)): raise ValueError("Workers not the same in all collections") + assert self.running.issuperset(self.idle.values()) 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 ws.status != Status.running: + assert ws.address not in self.idle if not ws.processing: assert not ws.occupancy - assert ws.address in self.idle + if ws.status == Status.running: + assert ws.address in self.idle assert (ws.status == Status.running) == (ws in self.running) for ws in self.running: @@ -4867,6 +4907,7 @@ def handle_worker_status_change( if ws.status == Status.running: self.running.add(ws) + self.check_idle_saturated(ws) recs = self.bulk_schedule_after_adding_worker(ws) if recs: client_msgs: dict = {} @@ -4875,6 +4916,7 @@ def handle_worker_status_change( self.send_all(client_msgs, worker_msgs) else: self.running.discard(ws) + self.idle.pop(ws.address, None) async def handle_request_refresh_who_has( self, keys: Iterable[str], worker: str, stimulus_id: str From ffbb53b976ab2f868ab1c94c7a7c1a4b9b16daca Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 14:39:37 -0600 Subject: [PATCH 32/98] Don't send queued tasks to no-worker A way more minimal fix than 5b9d825afb9ab3a61ab22afef3b047dde238bc5f, but not ideal because if only some workers are paused, we'll get root task overproduction on the others (because having `valid_workers` bypasses the root task detection logic). --- distributed/scheduler.py | 73 ++++++++++------------------------------ 1 file changed, 17 insertions(+), 56 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 0b3dc9d18d2..4487f117346 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1783,7 +1783,11 @@ def decide_worker( and not valid_workers and not ts.loose_restrictions ): - recommendations[ts.key] = "no-worker" + # FIXME when all workers are paused, root tasks in 'waiting' that could + # be queued would instead go to 'no-worker'. + if ts.state != "queued": + self.unrunnable.add(ts) + ts.state = "no-worker" return None # Group is larger than cluster with few dependencies? @@ -1795,12 +1799,10 @@ def decide_worker( and sum(map(len, tg.dependencies)) < 5 ): if math.isinf(self.WORKER_SATURATION): - pool = self.idle.values() if self.idle else self.running - if not pool: - recommendations[ts.key] = "no-worker" - return None - - return min(pool, key=lambda ws: len(ws.processing) / ws.nthreads) + return min( + (self.idle or self.workers).values(), + key=lambda ws: len(ws.processing) / ws.nthreads, + ) if not self.idle: # All workers busy? Task gets/stays queued. @@ -1820,20 +1822,12 @@ def decide_worker( ws, task_slots_available(ws, self.WORKER_SATURATION), ) - assert ws in self.running, (ws, self.running) return ws - if valid_workers is None and len(self.running) < len(self.workers): - if not self.running: - recommendations[ts.key] = "no-worker" - return None - - valid_workers = self.running - if ts.dependencies or valid_workers is not None: ws = decide_worker( ts, - self.running, + self.workers.values(), valid_workers, partial(self.worker_objective, ts), ) @@ -1861,7 +1855,6 @@ def decide_worker( if self.validate and ws is not None: assert ws.address in self.workers - assert ws in self.running, (ws, self.running) return ws @@ -2416,32 +2409,6 @@ def transition_waiting_queued(self, key, stimulus_id): pdb.set_trace() raise - def transition_waiting_no_worker(self, key, stimulus_id): - try: - ts: TaskState = self.tasks[key] - recommendations: dict = {} - client_msgs: dict = {} - worker_msgs: dict = {} - - if self.validate: - assert ts not in self.queued - assert not ts.who_has - assert not ts.exception_blame - assert not ts.processing_on - assert ts not in self.unrunnable - - ts.state = "no-worker" - self.unrunnable.add(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_queued_released(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] @@ -2592,7 +2559,6 @@ def transition_released_forgotten(self, key, stimulus_id): ("released", "waiting"): transition_released_waiting, ("waiting", "released"): transition_waiting_released, ("waiting", "processing"): transition_waiting_processing, - ("waiting", "no-worker"): transition_waiting_no_worker, ("waiting", "queued"): transition_waiting_queued, ("waiting", "memory"): transition_waiting_memory, ("queued", "released"): transition_queued_released, @@ -2681,8 +2647,7 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): if math.isinf(self.WORKER_SATURATION) else not worker_saturated(ws, self.WORKER_SATURATION) ): - if ws.status == Status.running: - idle[ws.address] = ws + idle[ws.address] = ws saturated.discard(ws) else: idle.pop(ws.address, None) @@ -2729,8 +2694,7 @@ def get_task_duration(self, ts: TaskState) -> float: def valid_workers(self, ts: TaskState) -> set: # set[WorkerState] | None """Return set of currently valid workers for key - If all workers are valid then this returns ``None``, in which case - any running worker can be used. + If all workers are valid then this returns ``None``. This checks tracks the following state: * worker_restrictions @@ -2779,7 +2743,10 @@ def valid_workers(self, ts: TaskState) -> set: # set[WorkerState] | None else: s &= ww - if s: + 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 @@ -4611,17 +4578,13 @@ def validate_state(self, allow_overlap: bool = False) -> None: if not (set(self.workers) == set(self.stream_comms)): raise ValueError("Workers not the same in all collections") - assert self.running.issuperset(self.idle.values()) 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 ws.status != Status.running: - assert ws.address not in self.idle if not ws.processing: assert not ws.occupancy - if ws.status == Status.running: - assert ws.address in self.idle + assert ws.address in self.idle assert (ws.status == Status.running) == (ws in self.running) for ws in self.running: @@ -4907,7 +4870,6 @@ def handle_worker_status_change( if ws.status == Status.running: self.running.add(ws) - self.check_idle_saturated(ws) recs = self.bulk_schedule_after_adding_worker(ws) if recs: client_msgs: dict = {} @@ -4916,7 +4878,6 @@ def handle_worker_status_change( self.send_all(client_msgs, worker_msgs) else: self.running.discard(ws) - self.idle.pop(ws.address, None) async def handle_request_refresh_who_has( self, keys: Iterable[str], worker: str, stimulus_id: str From 65735f809d04b83c2337abdbe1009c91d5acb0a1 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 15:12:30 -0600 Subject: [PATCH 33/98] Schedule rootish tasks when some workers are paused `valid_workers` will return a set if some workers are paused, even if the task doesn't have restrictions. This is anoying and a bit misleading, but possibly less intrusive of a change than 5b9d825afb9ab3a61ab22afef3b047dde238bc5f? --- distributed/scheduler.py | 23 ++++++++++++----- distributed/tests/test_scheduler.py | 40 +++++++++++++++++++---------- 2 files changed, 42 insertions(+), 21 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 4487f117346..daf1dd633cf 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1793,18 +1793,27 @@ def decide_worker( # Group is larger than cluster with few dependencies? # Minimize future data transfers. if ( - valid_workers is None + not ( + ts.worker_restrictions + or ts.host_restrictions + or ts.resource_restrictions + ) and len(tg) > self.total_nthreads * 2 and len(tg.dependencies) < 5 and sum(map(len, tg.dependencies)) < 5 ): if math.isinf(self.WORKER_SATURATION): return min( - (self.idle or self.workers).values(), + valid_workers or (self.idle or self.workers).values(), key=lambda ws: len(ws.processing) / ws.nthreads, ) - if not self.idle: + pool = ( + self.idle.values() + if valid_workers is None + else valid_workers.intersection(self.idle.values()) + ) + if not pool: # All workers busy? Task gets/stays queued. if self.validate: assert ts.key not in recommendations, (ts, recommendations[ts.key]) @@ -1814,9 +1823,7 @@ def decide_worker( # For root tasks, just pick the least busy worker. # NOTE: this will lead to worst-case scheduling with regards to co-assignment. - ws = min( - self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads - ) + ws = min(pool, key=lambda ws: len(ws.processing) / ws.nthreads) if self.validate: assert not worker_saturated(ws, self.WORKER_SATURATION), ( ws, @@ -2387,7 +2394,9 @@ def transition_waiting_queued(self, key, stimulus_id): if self.validate: assert ts not in self.queued - assert not self.idle, (ts, self.idle) + assert not ( + candidates := self.running.intersection(self.idle.values()) + ), (ts, candidates) # Copied from `transition_waiting_processing` assert not ts.waiting_on assert not ts.who_has diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 345a0e48fe3..fd5c109c2a6 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -272,24 +272,36 @@ async def test_root_task_overproduction(c, s, *nannies): Workload that would run out of memory and kill workers if >2 root tasks were ever in memory at once on a worker. """ - pids = [n.pid for n in nannies] + # Add a single paused worker just to throw off `valid_workers` + with dask.config.set({"distributed.worker.memory.pause": 0.0}): + async with Worker(s.address, nthreads=1) as paused: + while len(s.workers) != 3: + await asyncio.sleep(0.01) - @delayed(pure=True) # type: ignore - def big_data(size: int) -> str: - return "x" * size + while s.workers[paused.address].status != Status.paused: + await asyncio.sleep(0.01) + assert len(s.running) == 2 - roots = [ - big_data(parse_bytes("350 MiB"), dask_key_name=f"root-{i}") for i in range(16) - ] - passthrough = [delayed(slowidentity)(x) for x in roots] - memory_consumed = [delayed(len)(x) for x in passthrough] - reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] - final = sum(reduction) + pids = [n.pid for n in nannies] + + @delayed(pure=True) # type: ignore + def big_data(size: int) -> str: + return "x" * size + + roots = [ + big_data(parse_bytes("350 MiB"), dask_key_name=f"root-{i}") + for i in range(16) + ] + passthrough = [delayed(slowidentity)(x) for x in roots] + memory_consumed = [delayed(len)(x) for x in passthrough] + reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] + final = sum(reduction) - await c.compute(final) + await c.compute(final) - # No restarts - assert pids == [n.pid for n in nannies] + # No restarts + assert pids == [n.pid for n in nannies] + assert not paused.state.tasks @pytest.mark.parametrize("withhold", [True, False]) From 6bf710c844d81b70c4ee24c2a9cec76ff71f93dd Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 15:15:18 -0600 Subject: [PATCH 34/98] Rever less-intrusive all-paused handling I think overall excluding paused workers from idle is just more sensible. As is having `valid_workers` not concern itself with `running` workers. `valid_workers` should only deal in _task-specific_ restrictions, not restrictions that would apply to all tasks. This reverts commits 65735f809d04b83c2337abdbe1009c91d5acb0a1, ffbb53b976ab2f868ab1c94c7a7c1a4b9b16daca. --- distributed/scheduler.py | 94 +++++++++++++++++++---------- distributed/tests/test_scheduler.py | 40 +++++------- 2 files changed, 76 insertions(+), 58 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index daf1dd633cf..0b3dc9d18d2 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1783,37 +1783,26 @@ def decide_worker( and not valid_workers and not ts.loose_restrictions ): - # FIXME when all workers are paused, root tasks in 'waiting' that could - # be queued would instead go to 'no-worker'. - if ts.state != "queued": - self.unrunnable.add(ts) - ts.state = "no-worker" + recommendations[ts.key] = "no-worker" return None # Group is larger than cluster with few dependencies? # Minimize future data transfers. if ( - not ( - ts.worker_restrictions - or ts.host_restrictions - or ts.resource_restrictions - ) + valid_workers is None and len(tg) > self.total_nthreads * 2 and len(tg.dependencies) < 5 and sum(map(len, tg.dependencies)) < 5 ): if math.isinf(self.WORKER_SATURATION): - return min( - valid_workers or (self.idle or self.workers).values(), - key=lambda ws: len(ws.processing) / ws.nthreads, - ) + pool = self.idle.values() if self.idle else self.running + if not pool: + recommendations[ts.key] = "no-worker" + return None - pool = ( - self.idle.values() - if valid_workers is None - else valid_workers.intersection(self.idle.values()) - ) - if not pool: + return min(pool, key=lambda ws: len(ws.processing) / ws.nthreads) + + if not self.idle: # All workers busy? Task gets/stays queued. if self.validate: assert ts.key not in recommendations, (ts, recommendations[ts.key]) @@ -1823,18 +1812,28 @@ def decide_worker( # For root tasks, just pick the least busy worker. # NOTE: this will lead to worst-case scheduling with regards to co-assignment. - ws = min(pool, key=lambda ws: len(ws.processing) / ws.nthreads) + ws = min( + self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads + ) if self.validate: assert not worker_saturated(ws, self.WORKER_SATURATION), ( ws, task_slots_available(ws, self.WORKER_SATURATION), ) + assert ws in self.running, (ws, self.running) return ws + if valid_workers is None and len(self.running) < len(self.workers): + if not self.running: + recommendations[ts.key] = "no-worker" + return None + + valid_workers = self.running + if ts.dependencies or valid_workers is not None: ws = decide_worker( ts, - self.workers.values(), + self.running, valid_workers, partial(self.worker_objective, ts), ) @@ -1862,6 +1861,7 @@ def decide_worker( if self.validate and ws is not None: assert ws.address in self.workers + assert ws in self.running, (ws, self.running) return ws @@ -2394,9 +2394,7 @@ def transition_waiting_queued(self, key, stimulus_id): if self.validate: assert ts not in self.queued - assert not ( - candidates := self.running.intersection(self.idle.values()) - ), (ts, candidates) + assert not self.idle, (ts, self.idle) # Copied from `transition_waiting_processing` assert not ts.waiting_on assert not ts.who_has @@ -2418,6 +2416,32 @@ def transition_waiting_queued(self, key, stimulus_id): pdb.set_trace() raise + def transition_waiting_no_worker(self, key, stimulus_id): + try: + ts: TaskState = self.tasks[key] + recommendations: dict = {} + client_msgs: dict = {} + worker_msgs: dict = {} + + if self.validate: + assert ts not in self.queued + assert not ts.who_has + assert not ts.exception_blame + assert not ts.processing_on + assert ts not in self.unrunnable + + ts.state = "no-worker" + self.unrunnable.add(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_queued_released(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] @@ -2568,6 +2592,7 @@ def transition_released_forgotten(self, key, stimulus_id): ("released", "waiting"): transition_released_waiting, ("waiting", "released"): transition_waiting_released, ("waiting", "processing"): transition_waiting_processing, + ("waiting", "no-worker"): transition_waiting_no_worker, ("waiting", "queued"): transition_waiting_queued, ("waiting", "memory"): transition_waiting_memory, ("queued", "released"): transition_queued_released, @@ -2656,7 +2681,8 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): if math.isinf(self.WORKER_SATURATION) else not worker_saturated(ws, self.WORKER_SATURATION) ): - idle[ws.address] = ws + if ws.status == Status.running: + idle[ws.address] = ws saturated.discard(ws) else: idle.pop(ws.address, None) @@ -2703,7 +2729,8 @@ def get_task_duration(self, ts: TaskState) -> float: def valid_workers(self, ts: TaskState) -> set: # set[WorkerState] | None """Return set of currently valid workers for key - If all workers are valid then this returns ``None``. + If all workers are valid then this returns ``None``, in which case + any running worker can be used. This checks tracks the following state: * worker_restrictions @@ -2752,10 +2779,7 @@ def valid_workers(self, ts: TaskState) -> set: # set[WorkerState] | None else: s &= ww - if s is None: - if len(self.running) < len(self.workers): - return self.running.copy() - else: + if s: s = {self.workers[addr] for addr in s} if len(self.running) < len(self.workers): s &= self.running @@ -4587,13 +4611,17 @@ def validate_state(self, allow_overlap: bool = False) -> None: if not (set(self.workers) == set(self.stream_comms)): raise ValueError("Workers not the same in all collections") + assert self.running.issuperset(self.idle.values()) 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 ws.status != Status.running: + assert ws.address not in self.idle if not ws.processing: assert not ws.occupancy - assert ws.address in self.idle + if ws.status == Status.running: + assert ws.address in self.idle assert (ws.status == Status.running) == (ws in self.running) for ws in self.running: @@ -4879,6 +4907,7 @@ def handle_worker_status_change( if ws.status == Status.running: self.running.add(ws) + self.check_idle_saturated(ws) recs = self.bulk_schedule_after_adding_worker(ws) if recs: client_msgs: dict = {} @@ -4887,6 +4916,7 @@ def handle_worker_status_change( self.send_all(client_msgs, worker_msgs) else: self.running.discard(ws) + self.idle.pop(ws.address, None) async def handle_request_refresh_who_has( self, keys: Iterable[str], worker: str, stimulus_id: str diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index fd5c109c2a6..345a0e48fe3 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -272,36 +272,24 @@ async def test_root_task_overproduction(c, s, *nannies): Workload that would run out of memory and kill workers if >2 root tasks were ever in memory at once on a worker. """ - # Add a single paused worker just to throw off `valid_workers` - with dask.config.set({"distributed.worker.memory.pause": 0.0}): - async with Worker(s.address, nthreads=1) as paused: - while len(s.workers) != 3: - await asyncio.sleep(0.01) - - while s.workers[paused.address].status != Status.paused: - await asyncio.sleep(0.01) - assert len(s.running) == 2 + pids = [n.pid for n in nannies] - pids = [n.pid for n in nannies] - - @delayed(pure=True) # type: ignore - def big_data(size: int) -> str: - return "x" * size + @delayed(pure=True) # type: ignore + def big_data(size: int) -> str: + return "x" * size - roots = [ - big_data(parse_bytes("350 MiB"), dask_key_name=f"root-{i}") - for i in range(16) - ] - passthrough = [delayed(slowidentity)(x) for x in roots] - memory_consumed = [delayed(len)(x) for x in passthrough] - reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] - final = sum(reduction) + roots = [ + big_data(parse_bytes("350 MiB"), dask_key_name=f"root-{i}") for i in range(16) + ] + passthrough = [delayed(slowidentity)(x) for x in roots] + memory_consumed = [delayed(len)(x) for x in passthrough] + reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] + final = sum(reduction) - await c.compute(final) + await c.compute(final) - # No restarts - assert pids == [n.pid for n in nannies] - assert not paused.state.tasks + # No restarts + assert pids == [n.pid for n in nannies] @pytest.mark.parametrize("withhold", [True, False]) From 25e6f3baf288ea4af28b52e9e6548fd949e68753 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 15:16:44 -0600 Subject: [PATCH 35/98] Decrease test_root_task_overproduction size Workers seem to be running out of memory on CI. Probably different base unmanaged memory sizes than my machine. This is tricky. --- distributed/tests/test_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 345a0e48fe3..b53fa500d8b 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -279,7 +279,7 @@ def big_data(size: int) -> str: return "x" * size roots = [ - big_data(parse_bytes("350 MiB"), dask_key_name=f"root-{i}") for i in range(16) + big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(16) ] passthrough = [delayed(slowidentity)(x) for x in roots] memory_consumed = [delayed(len)(x) for x in passthrough] From b86fe0f61c4f7c9c63151f6a3630dab3db3c9540 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 15:47:20 -0600 Subject: [PATCH 36/98] Co-assignment when queuing is disabled --- distributed/scheduler.py | 35 ++++++++++++++++++++++++++--- distributed/tests/test_scheduler.py | 7 ++++-- 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 9be831f8d16..adc3af81946 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -804,6 +804,14 @@ class TaskGroup: #: 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 @@ -823,6 +831,8 @@ def __init__(self, name: str): 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 @@ -1331,7 +1341,7 @@ def __init__( self.unrunnable = unrunnable self.validate = validate self.workers = workers - self.running = { + self.running: set[WorkerState] = { 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} @@ -1791,13 +1801,32 @@ def decide_worker( and len(tg.dependencies) < 5 and sum(map(len, tg.dependencies)) < 5 ): - if math.isinf(self.WORKER_SATURATION): + if math.isinf(self.WORKER_SATURATION): # no scheduler-side queuing pool = self.idle.values() if self.idle else self.running if not pool: recommendations[ts.key] = "no-worker" return None - return min(pool, key=lambda ws: len(ws.processing) / ws.nthreads) + lws = tg.last_worker + if not ( + lws + and tg.last_worker_tasks_left + and self.workers.get(lws.address) is lws + ): + # Last-used worker is full or unknown; pick a new worker for the next few tasks + ws = min(pool, key=partial(self.worker_objective, ts)) + tg.last_worker_tasks_left = math.floor( + (len(tg) / self.total_nthreads) * ws.nthreads + ) + else: + ws = lws + + # 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 not self.idle: # All workers busy? Task gets/stays queued. diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index b53fa500d8b..509cad37124 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -134,7 +134,7 @@ async def test_decide_worker_with_restrictions(client, s, a, b, c): assert x.key in a.data or x.key in b.data -@pytest.mark.skip("Current queuing does not support co-assignment") +# @pytest.mark.skip("Current queuing does not support co-assignment") @pytest.mark.parametrize("ndeps", [0, 1, 4]) @pytest.mark.parametrize( "nthreads", @@ -147,7 +147,10 @@ def test_decide_worker_coschedule_order_neighbors(ndeps, nthreads): @gen_cluster( client=True, nthreads=nthreads, - config={"distributed.scheduler.work-stealing": False}, + config={ + "distributed.scheduler.work-stealing": False, + "distributed.scheduler.worker-saturation": float("inf"), + }, scheduler_kwargs=dict( # TODO remove dashboard=True, dashboard_address=":8787", From 7ebd1d99240448eebc346370ce73dece78c92e0d Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 16:18:39 -0600 Subject: [PATCH 37/98] Fix co-assignment for binary operations Bit of a hack, but closes https://github.com/dask/distributed/issues/6597. I'd like to have a better metric for the batch size, but I think this is about as good as we can get. Any reasonably large number will do here. --- distributed/scheduler.py | 31 +++++++++-------------- distributed/tests/test_scheduler.py | 39 ++++++++++++++++------------- 2 files changed, 33 insertions(+), 37 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index adc3af81946..f0393485334 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -804,14 +804,6 @@ class TaskGroup: #: 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 @@ -831,8 +823,6 @@ def __init__(self, name: str): 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 @@ -1269,6 +1259,8 @@ class SchedulerState: "extensions", "host_info", "idle", + "last_root_worker", + "last_root_worker_tasks_left", "n_tasks", "queued", "resources", @@ -1337,6 +1329,8 @@ def __init__( self.total_nthreads = 0 self.total_occupancy = 0.0 self.unknown_durations: dict[str, set[TaskState]] = {} + self.last_root_worker: WorkerState | None = None + self.last_root_worker_tasks_left: int = 0 self.queued = queued self.unrunnable = unrunnable self.validate = validate @@ -1807,25 +1801,24 @@ def decide_worker( recommendations[ts.key] = "no-worker" return None - lws = tg.last_worker + lws = self.last_root_worker if not ( lws - and tg.last_worker_tasks_left + and self.last_root_worker_tasks_left and self.workers.get(lws.address) is lws ): # Last-used worker is full or unknown; pick a new worker for the next few tasks - ws = min(pool, key=partial(self.worker_objective, ts)) - tg.last_worker_tasks_left = math.floor( + ws = self.last_root_worker = min( + pool, key=lambda ws: len(ws.processing) / ws.nthreads + ) + # TODO better batching metric (`len(tg)` is not necessarily the total number of root tasks!) + self.last_root_worker_tasks_left = math.floor( (len(tg) / self.total_nthreads) * ws.nthreads ) else: ws = lws - # 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 + self.last_root_worker_tasks_left -= 1 return ws if not self.idle: diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 509cad37124..a4fbf6242ca 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -134,7 +134,6 @@ async def test_decide_worker_with_restrictions(client, s, a, b, c): assert x.key in a.data or x.key in b.data -# @pytest.mark.skip("Current queuing does not support co-assignment") @pytest.mark.parametrize("ndeps", [0, 1, 4]) @pytest.mark.parametrize( "nthreads", @@ -151,10 +150,6 @@ def test_decide_worker_coschedule_order_neighbors(ndeps, nthreads): "distributed.scheduler.work-stealing": False, "distributed.scheduler.worker-saturation": float("inf"), }, - scheduler_kwargs=dict( # TODO remove - dashboard=True, - dashboard_address=":8787", - ), ) async def test_decide_worker_coschedule_order_neighbors_(c, s, *workers): r""" @@ -254,6 +249,24 @@ def random(**kwargs): test_decide_worker_coschedule_order_neighbors_() +@pytest.mark.parametrize("ngroups", [1, 2, 3, 5]) +@gen_cluster( + client=True, + nthreads=[("", 1), ("", 1)], + config={ + "distributed.scheduler.worker-saturation": float("inf"), + }, +) +async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): + roots = [[delayed(i, name=f"x-{n}-{i}") for i in range(8)] for n in range(ngroups)] + zs = [sum(rs) for rs in zip(*roots)] + + await c.gather(c.compute(zs)) + + assert not a.incoming_transfer_log, [l["keys"] for l in a.incoming_transfer_log] + assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] + + @pytest.mark.slow @gen_cluster( client=True, @@ -381,17 +394,7 @@ async def _test_saturation_factor(c, s, a, b): @pytest.mark.skip("Current queuing does not support co-assignment") -@pytest.mark.parametrize( - "saturation_factor", - [ - 1.0, - 2.0, - pytest.param( - float("inf"), - marks=pytest.mark.skip("https://github.com/dask/distributed/issues/6597"), - ), - ], -) +@pytest.mark.parametrize("saturation_factor", [1.0, 2.0, float("inf")]) @gen_cluster( client=True, nthreads=[("", 2), ("", 1)], @@ -406,8 +409,8 @@ async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor assert not a.incoming_transfer_log, [l["keys"] for l in a.incoming_transfer_log] assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] - assert len(a.tasks) == 18 - assert len(b.tasks) == 9 + assert len(a.state.tasks) == 18 + assert len(b.state.tasks) == 9 @pytest.mark.slow From 034f980b85402ce77193505741eddc2a89b565b6 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 23 Jun 2022 19:44:09 -0600 Subject: [PATCH 38/98] Turn withholding off by default Want to see if CI passes. This would be retaining current scheduling behavior. Task withholding would be behind a feature flag. --- distributed/distributed.yaml | 2 +- distributed/tests/test_scheduler.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index 124e001b0c4..e88eb191ff8 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -22,7 +22,7 @@ distributed: events-log-length: 100000 work-stealing: True # workers should steal tasks from each other work-stealing-interval: 100ms # Callback time for work stealing - worker-saturation: 1.0 # Send this fraction of nthreads root tasks to workers + worker-saturation: .inf # Send this fraction of nthreads root tasks to workers worker-ttl: "5 minutes" # like '60s'. Time to live for workers. They must heartbeat faster than this pickle: True # Is the scheduler allowed to deserialize arbitrary bytestrings preload: [] # Run custom modules with Scheduler diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index a4fbf6242ca..e8c1113e583 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -148,7 +148,6 @@ def test_decide_worker_coschedule_order_neighbors(ndeps, nthreads): nthreads=nthreads, config={ "distributed.scheduler.work-stealing": False, - "distributed.scheduler.worker-saturation": float("inf"), }, ) async def test_decide_worker_coschedule_order_neighbors_(c, s, *workers): @@ -253,9 +252,6 @@ def random(**kwargs): @gen_cluster( client=True, nthreads=[("", 1), ("", 1)], - config={ - "distributed.scheduler.worker-saturation": float("inf"), - }, ) async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): roots = [[delayed(i, name=f"x-{n}-{i}") for i in range(8)] for n in range(ngroups)] @@ -278,6 +274,7 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): dashboard_address=":8787", ), config={ + "distributed.scheduler.worker-saturation": 1.0, "distributed.worker.memory.target": False, "distributed.worker.memory.spill": False, "distributed.scheduler.work-stealing": False, @@ -319,6 +316,7 @@ def big_data(size: int) -> str: dashboard_address=":8787", ), config={ + "distributed.scheduler.worker-saturation": 1.0, # With typical overhead, 1 task can be in memory but the second will trigger a pause "distributed.worker.memory.pause": 0.4, "distributed.worker.memory.target": False, @@ -398,6 +396,7 @@ async def _test_saturation_factor(c, s, a, b): @gen_cluster( client=True, nthreads=[("", 2), ("", 1)], + config={"distributed.scheduler.worker-saturation": 1.0}, ) async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor): s.WORKER_SATURATION = saturation_factor @@ -421,6 +420,7 @@ async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor dashboard=True, dashboard_address=":8787", ), + config={"distributed.scheduler.worker-saturation": 1.0}, ) async def test_queued_tasks_rebalance(client, s, a, b): """ From a63d25b35f20f85b81536fbb311f66e950b5be5f Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 16:29:25 -0600 Subject: [PATCH 39/98] Remove redundant insert into `idle` Already covered by `if p < nc` in `check_idle_saturated`. But the one removed here didn't check for `status == Status.running` --- distributed/scheduler.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 81679b2603e..36e275949c3 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -3875,9 +3875,6 @@ async def add_worker( 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) From 5f7e7f1492aa9600f10a4dfae059a59af6679e4e Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 16:30:20 -0600 Subject: [PATCH 40/98] Update `idle` docstring --- distributed/scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 36e275949c3..714e4a13284 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1256,7 +1256,7 @@ class SchedulerState: * **workers:** ``{worker key: WorkerState}`` Workers currently connected to the scheduler * **idle:** ``{WorkerState}``: - Set of workers that are not fully utilized + Set of workers that are currently in running state and not fully utilized * **saturated:** ``{WorkerState}``: Set of workers that are not over-utilized * **running:** ``{WorkerState}``: From 9aeecc9946515e28dda8982fe69d502ca65f4f6d Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 16:41:49 -0600 Subject: [PATCH 41/98] fix `test_ready_remove_worker` --- distributed/tests/test_scheduler.py | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 4dcb898af01..e1baba49d67 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -3,6 +3,7 @@ import asyncio import json import logging +import math import operator import pickle import re @@ -824,7 +825,17 @@ async def test_ready_remove_worker(s, a, b): dependencies={"x-%d" % i: [] for i in range(20)}, ) - assert all(len(w.processing) == w.nthreads for w in s.workers.values()) + if s.WORKER_SATURATION == 1: + cmp = operator.eq + elif math.isinf(s.WORKER_SATURATION): + cmp = operator.gt + else: + pytest.fail(f"{s.WORKER_OVERSATURATION=}, must be 1 or inf") + + assert all(cmp(len(w.processing), w.nthreads) for w in s.workers.values()), ( + list(s.workers.values()), + s.WORKER_SATURATION, + ) assert sum(len(w.processing) for w in s.workers.values()) + len(s.queued) == len( s.tasks ) @@ -832,7 +843,10 @@ async def test_ready_remove_worker(s, a, b): await s.remove_worker(address=a.address, stimulus_id="test") assert set(s.workers) == {b.address} - assert all(len(w.processing) == w.nthreads for w in s.workers.values()) + assert all(cmp(len(w.processing), w.nthreads) for w in s.workers.values()), ( + list(s.workers.values()), + s.WORKER_SATURATION, + ) assert sum(len(w.processing) for w in s.workers.values()) + len(s.queued) == len( s.tasks ) From dcb11e4d82ce4f71e0024f47f1b01929d46f573c Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 17:26:13 -0600 Subject: [PATCH 42/98] fix `test_saturation_factor` Not sure why these numbers changed --- distributed/tests/test_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index e1baba49d67..ffee23ee14f 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -359,7 +359,7 @@ def big_data(size: int) -> str: (2.0, (4, 2)), (1.0, (2, 1)), (-1.0, (1, 1)), - (float("inf"), (7, 3)) + (float("inf"), (6, 4)) # ^ depends on root task assignment logic; ok if changes, just needs to add up to 10 ], ) From 7dfc83e8e2b2b7da774436dcd4e280480cbd8987 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 17:29:20 -0600 Subject: [PATCH 43/98] remove debug dashboards from tests --- distributed/tests/test_scheduler.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index ffee23ee14f..c20f112b7a4 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -270,10 +270,6 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): nthreads=[("", 2)] * 2, worker_kwargs={"memory_limit": "1.0GiB"}, Worker=Nanny, - scheduler_kwargs=dict( # TODO remove - dashboard=True, - dashboard_address=":8787", - ), config={ "distributed.scheduler.worker-saturation": 1.0, "distributed.worker.memory.target": False, @@ -312,10 +308,6 @@ def big_data(size: int) -> str: nthreads=[("", 2)] * 2, worker_kwargs={"memory_limit": "1.0 GiB"}, Worker=Nanny, - scheduler_kwargs=dict( # TODO remove - dashboard=True, - dashboard_address=":8787", - ), config={ "distributed.scheduler.worker-saturation": 1.0, # With typical overhead, 1 task can be in memory but the second will trigger a pause @@ -417,10 +409,6 @@ async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor @gen_cluster( client=True, nthreads=[("", 2)] * 2, - scheduler_kwargs=dict( # TODO remove - dashboard=True, - dashboard_address=":8787", - ), config={"distributed.scheduler.worker-saturation": 1.0}, ) async def test_queued_tasks_rebalance(client, s, a, b): From c99bbe8318e1eb179e663cf37da066266a0ccfb4 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 18:15:53 -0600 Subject: [PATCH 44/98] fix `progress_stream` --- distributed/diagnostics/progress_stream.py | 2 +- distributed/diagnostics/tests/test_progress_stream.py | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/distributed/diagnostics/progress_stream.py b/distributed/diagnostics/progress_stream.py index 6172bff1263..764aef790ea 100644 --- a/distributed/diagnostics/progress_stream.py +++ b/distributed/diagnostics/progress_stream.py @@ -17,7 +17,7 @@ def counts(scheduler, allprogress): {"all": valmap(len, allprogress.all), "nbytes": allprogress.nbytes}, { state: valmap(len, allprogress.state[state]) - for state in ["memory", "erred", "released", "processing"] + for state in ["memory", "erred", "released", "processing", "queued"] }, ) diff --git a/distributed/diagnostics/tests/test_progress_stream.py b/distributed/diagnostics/tests/test_progress_stream.py index 73a5be81abe..49c93b212e2 100644 --- a/distributed/diagnostics/tests/test_progress_stream.py +++ b/distributed/diagnostics/tests/test_progress_stream.py @@ -18,6 +18,7 @@ def test_progress_quads(): "erred": {"inc": 0, "dec": 1, "add": 0}, "released": {"inc": 1, "dec": 0, "add": 1}, "processing": {"inc": 1, "dec": 0, "add": 2}, + "queued": {"inc": 1, "dec": 0, "add": 2}, } d = progress_quads(msg, nrows=2) @@ -35,11 +36,13 @@ def test_progress_quads(): "memory": [2, 1, 0], "erred": [0, 0, 1], "processing": [1, 2, 0], + "queued": [1, 2, 0], "done": ["3 / 5", "2 / 4", "1 / 1"], "released-loc": [0.9 * 1 / 5, 0.25 * 0.9, 1.0], "memory-loc": [0.9 * 3 / 5, 0.5 * 0.9, 1.0], "erred-loc": [0.9 * 3 / 5, 0.5 * 0.9, 1.9], "processing-loc": [0.9 * 4 / 5, 1 * 0.9, 1 * 0.9 + 1], + "queued-loc": [1 * 0.9, 1.5 * 0.9, 1 * 0.9 + 1], } assert d == expected @@ -52,6 +55,7 @@ def test_progress_quads_too_many(): "erred": {k: 0 for k in keys}, "released": {k: 0 for k in keys}, "processing": {k: 0 for k in keys}, + "queued": {k: 0 for k in keys}, } d = progress_quads(msg, nrows=6, ncols=3) @@ -78,6 +82,7 @@ async def test_progress_stream(c, s, a, b): "memory": {"div": 9, "inc": 1}, "released": {"inc": 4}, "processing": {}, + "queued": {}, } assert set(nbytes) == set(msg["all"]) assert all(v > 0 for v in nbytes.values()) @@ -95,6 +100,7 @@ def test_progress_quads_many_functions(): "erred": {fn: 0 for fn in funcnames}, "released": {fn: 0 for fn in funcnames}, "processing": {fn: 0 for fn in funcnames}, + "queued": {fn: 0 for fn in funcnames}, } d = progress_quads(msg, nrows=2) From c1544f314c3e2626170c40d3ce57a87f8f4230dc Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 18:17:00 -0600 Subject: [PATCH 45/98] fix `test_prometheus_collect_task_states` --- .../http/scheduler/tests/test_scheduler_http.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/distributed/http/scheduler/tests/test_scheduler_http.py b/distributed/http/scheduler/tests/test_scheduler_http.py index c2d75d8206f..158384b7587 100644 --- a/distributed/http/scheduler/tests/test_scheduler_http.py +++ b/distributed/http/scheduler/tests/test_scheduler_http.py @@ -137,7 +137,15 @@ async def fetch_metrics(): ] return active_metrics, forgotten_tasks - expected = {"memory", "released", "processing", "waiting", "no-worker", "erred"} + expected = { + "memory", + "released", + "queued", + "processing", + "waiting", + "no-worker", + "erred", + } # Ensure that we get full zero metrics for all states even though the # scheduler did nothing, yet From 349712f63d55b597851d077d7fe9684215915d92 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 18:19:25 -0600 Subject: [PATCH 46/98] fix config json schema --- distributed/distributed-schema.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 3577dd47a16..e00ffa89981 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -118,7 +118,7 @@ properties: How frequently to balance worker loads worker-saturation: - type: float + type: number description: | Controls how many root tasks are sent to workers (like a `readahead`). From 594585e7a33ac44067e2b95cf68ccb3e56d0bffc Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 18:42:23 -0600 Subject: [PATCH 47/98] fix retire workers --- distributed/scheduler.py | 6 +++++- distributed/tests/test_active_memory_manager.py | 2 ++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 714e4a13284..af280f0ec18 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -4700,7 +4700,10 @@ def validate_state(self, allow_overlap: bool = False) -> None: if not (set(self.workers) == set(self.stream_comms)): raise ValueError("Workers not the same in all collections") - assert self.running.issuperset(self.idle.values()) + assert self.running.issuperset(self.idle.values()), ( + self.running, + list(self.idle.values()), + ) for w, ws in self.workers.items(): assert isinstance(w, str), (type(w), w) assert isinstance(ws, WorkerState), (type(ws), ws) @@ -6333,6 +6336,7 @@ async def retire_workers( prev_status = ws.status ws.status = Status.closing_gracefully self.running.discard(ws) + self.idle.pop(ws.address, None) # 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( diff --git a/distributed/tests/test_active_memory_manager.py b/distributed/tests/test_active_memory_manager.py index 10c8d9da9cc..dffe85d24dc 100644 --- a/distributed/tests/test_active_memory_manager.py +++ b/distributed/tests/test_active_memory_manager.py @@ -782,6 +782,8 @@ async def test_RetireWorker_no_remove(c, s, a, b): while s.tasks["x"].who_has != {s.workers[b.address]}: await asyncio.sleep(0.01) assert a.address in s.workers + assert a.status == Status.closing_gracefully + assert s.workers[a.address].status == Status.closing_gracefully # Policy has been removed without waiting for worker to disappear from # Scheduler.workers assert not s.extensions["amm"].policies From b4f843d457dfc1ea044b7e949ec66342366968cd Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 17:40:26 -0600 Subject: [PATCH 48/98] update `validate_task_state` --- distributed/scheduler.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index af280f0ec18..fb424f7e6e4 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7844,7 +7844,7 @@ def validate_task_state(ts: TaskState) -> None: str(dts), str(dts.dependents), ) - if ts.state in ("waiting", "processing", "no-worker"): + if ts.state in ("waiting", "queued", "processing", "no-worker"): assert dts in ts.waiting_on or dts.who_has, ( "dep missing", str(ts), @@ -7853,7 +7853,7 @@ def validate_task_state(ts: TaskState) -> None: assert dts.state != "forgotten" for dts in ts.waiters: - assert dts.state in ("waiting", "processing", "no-worker"), ( + assert dts.state in ("waiting", "queued", "processing", "no-worker"), ( "waiter not in play", str(ts), str(dts), From 2db4db99f69e35d8c0d543cbe84a12e39ea759ab Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 17 Aug 2022 17:59:49 -0600 Subject: [PATCH 49/98] fix `test_saturation_factor` again Apparently they're just unpredictable --- distributed/tests/test_scheduler.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index c20f112b7a4..45453fcb4a4 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -375,8 +375,12 @@ async def _test_saturation_factor(c, s, a, b): ) or b.state.executing_count < min(b.state.nthreads, expected_task_counts[1]): await asyncio.sleep(0.01) - assert len(a.state.tasks) == expected_task_counts[0] - assert len(b.state.tasks) == expected_task_counts[1] + if math.isfinite(saturation): + assert len(a.state.tasks) == expected_task_counts[0] + assert len(b.state.tasks) == expected_task_counts[1] + else: + # Assignment is nondeterministic for some reason without queuing + assert len(a.state.tasks) > len(b.state.tasks) await event.set() await c.gather(fs) From 8395ef4acc888f84f2f5bc48521301f7c1a8ae94 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 18 Aug 2022 18:54:28 -0600 Subject: [PATCH 50/98] ignore long-running in `task_slots_available` --- distributed/scheduler.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index fb424f7e6e4..b103af7cfb5 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -4710,6 +4710,7 @@ def validate_state(self, allow_overlap: bool = False) -> None: assert ws.address == w if ws.status != Status.running: assert ws.address not in self.idle + assert ws.long_running.issubset(ws.processing) if not ws.processing: assert not ws.occupancy if ws.status == Status.running: @@ -7978,7 +7979,9 @@ def task_slots_available(ws: WorkerState, saturation_factor: float) -> int: "Number of tasks that can be sent to this worker without oversaturating it" assert not math.isinf(saturation_factor) nthreads = ws.nthreads - return max(int(saturation_factor * nthreads), 1) - len(ws.processing) + return max(int(saturation_factor * nthreads), 1) - ( + len(ws.processing) - len(ws.long_running) + ) def worker_saturated(ws: WorkerState, saturation_factor: float) -> bool: From 36a60a5e358ea2a5d16597651126ac5892203b01 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 18 Aug 2022 19:36:07 -0600 Subject: [PATCH 51/98] hackily consider queue in adaptive target TODO this is one of the main things unhandled in this PR: how do we address occupancy? Do queued tasks contribute to total occupancy or not? In either case, how is that implemented?? (I don't really want to make a `queued_occ` dict tracking per-task occupancy, like we have for processing; that feels like overkill.) --- distributed/scheduler.py | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index b103af7cfb5..5b29246e723 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7402,19 +7402,29 @@ def adaptive_target(self, target_duration=None): target_duration = parse_timedelta(target_duration) # CPU + + # FIXME maintain a proper estimate of queued occupancy!! + # This is merely a hack intended to make queuing sorta work with adaptive scaling + # so people can try it out in the short term (at least the cluster should scale up + # when tasks are queued). + avg_duration = ( + (self.total_occupancy / self.total_nthreads) if self.total_nthreads else 0 + ) + queued_occupancy = len(self.queued) * avg_duration + cpu = math.ceil( - self.total_occupancy / target_duration + (self.total_occupancy + queued_occupancy) / target_duration ) # TODO: threads per worker # Avoid a few long tasks from asking for many cores - tasks_processing = len(self.queued) + tasks_ready = len(self.queued) for ws in self.workers.values(): - tasks_processing += len(ws.processing) + tasks_ready += len(ws.processing) - if tasks_processing > cpu: + if tasks_ready > cpu: break else: - cpu = min(tasks_processing, cpu) + cpu = min(tasks_ready, cpu) if self.unrunnable and not self.workers: cpu = max(1, cpu) From da044389a50cf659e5666024f3f75aabd096263e Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 18 Aug 2022 19:53:43 -0600 Subject: [PATCH 52/98] update explanation in schema slightly --- distributed/distributed-schema.yaml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index e00ffa89981..057d251edb9 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -122,18 +122,18 @@ properties: description: | Controls how many root tasks are sent to workers (like a `readahead`). - `max(floor(worker-oversaturation * worker.nthreads), 1)` root tasks are sent to a + `max(floor(worker-saturation * worker.nthreads), 1)` root tasks are sent to a worker at a time. If `.inf`, all runnable tasks are immediately sent to workers. Allowing oversaturation (> 1.0) means a worker will start running a new root task as soon as it completes the previous, even if there is a higher-priority downstream task to run. This reduces worker idleness, by letting workers do something while waiting for - further instructions from the scheduler. + further instructions from the scheduler, even if it's not the most useful thing. This generally comes at the expense of increased memory usage. It leads to "wider" (more breadth-first) execution of the graph. - Compute-bound workloads benefit from oversaturation. Memory-bound workloads should + Compute-bound workloads may benefit from oversaturation. Memory-bound workloads should generally leave `worker-saturation` at 1.0, though 1.25-1.5 could slightly improve performance if ample memory is available. From c92236cad8acdd974b98583ca93ba423874bcc79 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 18 Aug 2022 20:06:53 -0600 Subject: [PATCH 53/98] update comments in `decide_worker` --- distributed/scheduler.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 5b29246e723..83631fc4c9e 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1811,14 +1811,17 @@ def decide_worker( return None # Group is larger than cluster with few dependencies? - # Minimize future data transfers. + # This is a root task. + # Queue it, or if queuing is disabled, schedule near previous root task + # to 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 ): - if math.isinf(self.WORKER_SATURATION): # no scheduler-side queuing + # Queuing disabled case + if math.isinf(self.WORKER_SATURATION): pool = self.idle.values() if self.idle else self.running if not pool: recommendations[ts.key] = "no-worker" @@ -1844,6 +1847,7 @@ def decide_worker( self.last_root_worker_tasks_left -= 1 return ws + # Queuing enabled case if not self.idle: # All workers busy? Task gets/stays queued. if self.validate: @@ -1870,6 +1874,7 @@ def decide_worker( recommendations[ts.key] = "no-worker" return None + # If there were no restrictions, `valid_workers()` valid_workers = self.running if ts.dependencies or valid_workers is not None: @@ -2779,7 +2784,9 @@ 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``, in which case - any running worker can be used. + any *running* worker can be used. + Otherwise, the subset of running workers valid for this task + is returned. This checks tracks the following state: * worker_restrictions From e990b92318bf134801b4d3f9e7017fd21b73a109 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 18 Aug 2022 20:17:26 -0600 Subject: [PATCH 54/98] formalize `transition_queued_processing` assertion --- distributed/scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 83631fc4c9e..44cab1365de 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2527,7 +2527,7 @@ def transition_queued_processing(self, key, stimulus_id): worker_msgs: dict = {} if self.validate: - assert not ts.actor, "Actors can't be queued wat" + assert not ts.actor, f"Actors can't be queued: {ts}" assert ts in self.queued _add_to_processing(self, ts, recommendations, worker_msgs) From 0d21c7860888bae72e6ce20c38f93bc34b0454e5 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 18 Aug 2022 20:34:33 -0600 Subject: [PATCH 55/98] correct bulk_schedule comment I mistakenly thought that in the transitions loop, new recommendations were processed after old ones. I believe it's the opposite (`dict.update` will add the new items at the end, `dict.popitem` will pop those new items off the end). It wouldn't be too hard to sort all the recommendations here, just some extra allocations and copies. --- distributed/scheduler.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 44cab1365de..2aea2068b2c 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2953,13 +2953,11 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): valid = self.valid_workers(ts) if valid is None or ws in valid: now_runnable.append(ts) - # These recommendations will generate {"op": "compute-task"} messages - # to the worker in reversed order + # Recommendations are processed LIFO, hence the reversed order now_runnable.sort(key=operator.attrgetter("priority"), reverse=True) for ts in now_runnable: - # TODO queued tasks will take precedence over newly-runnable tasks - # since waiting->processing will put them at the end of the transitions - # dict (since it's a `popitem` then `update`). Is that okay? + # TODO this assumes unrunnable tasks are always higher-priority than queued + # tasks. Should we sort all of the recommendations instead? recommendations[ts.key] = "waiting" return recommendations From b40cec1b74f7e54bc2a5b63f5f9a2594aed452d4 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 23 Aug 2022 11:59:57 -0600 Subject: [PATCH 56/98] private `_task_slots_available` --- distributed/scheduler.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 2aea2068b2c..f03a7c76b07 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1864,7 +1864,7 @@ def decide_worker( if self.validate: assert not worker_saturated(ws, self.WORKER_SATURATION), ( ws, - task_slots_available(ws, self.WORKER_SATURATION), + _task_slots_available(ws, self.WORKER_SATURATION), ) assert ws in self.running, (ws, self.running) return ws @@ -2939,7 +2939,9 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): # Schedule any queued tasks onto the new worker if not math.isinf(self.WORKER_SATURATION) and self.queued: for qts in reversed( - list(self.queued.topk(task_slots_available(ws, self.WORKER_SATURATION))) + list( + self.queued.topk(_task_slots_available(ws, self.WORKER_SATURATION)) + ) ): if self.validate: assert qts.state == "queued" @@ -7990,7 +7992,7 @@ def heartbeat_interval(n: int) -> float: return n / 200 + 1 -def task_slots_available(ws: WorkerState, saturation_factor: float) -> int: +def _task_slots_available(ws: WorkerState, saturation_factor: float) -> int: "Number of tasks that can be sent to this worker without oversaturating it" assert not math.isinf(saturation_factor) nthreads = ws.nthreads @@ -8002,7 +8004,7 @@ def task_slots_available(ws: WorkerState, saturation_factor: float) -> int: def worker_saturated(ws: WorkerState, saturation_factor: float) -> bool: if math.isinf(saturation_factor): return False - return task_slots_available(ws, saturation_factor) <= 0 + return _task_slots_available(ws, saturation_factor) <= 0 class KilledWorker(Exception): From 12b94d0552adc66eb9d88cdc6e5d70d842d4645e Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 23 Aug 2022 12:00:30 -0600 Subject: [PATCH 57/98] `worker_saturated` -> `_worker_full` --- distributed/scheduler.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index f03a7c76b07..e8df96d5bb1 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1862,7 +1862,7 @@ def decide_worker( self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads ) if self.validate: - assert not worker_saturated(ws, self.WORKER_SATURATION), ( + assert not _worker_full(ws, self.WORKER_SATURATION), ( ws, _task_slots_available(ws, self.WORKER_SATURATION), ) @@ -2733,7 +2733,7 @@ def check_idle_saturated(self, ws: WorkerState, occ: float = -1.0): if ( (p < nc or occ < nc * avg / 2) if math.isinf(self.WORKER_SATURATION) - else not worker_saturated(ws, self.WORKER_SATURATION) + else not _worker_full(ws, self.WORKER_SATURATION) ): if ws.status == Status.running: idle[ws.address] = ws @@ -7567,7 +7567,7 @@ def _remove_from_processing( state.release_resources(ts, ws) # If a slot has opened up for a queued task, schedule it. - if state.queued and not worker_saturated(ws, state.WORKER_SATURATION): + if state.queued and not _worker_full(ws, state.WORKER_SATURATION): qts = state.queued.peek() if state.validate: assert qts.state == "queued", qts.state @@ -8001,7 +8001,7 @@ def _task_slots_available(ws: WorkerState, saturation_factor: float) -> int: ) -def worker_saturated(ws: WorkerState, saturation_factor: float) -> bool: +def _worker_full(ws: WorkerState, saturation_factor: float) -> bool: if math.isinf(saturation_factor): return False return _task_slots_available(ws, saturation_factor) <= 0 From 4c0e768ac42e06691ee83eef9186222fdfa5c2bf Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 23 Aug 2022 12:30:13 -0600 Subject: [PATCH 58/98] remove unused `_add_to_processing` return value --- distributed/scheduler.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index e8df96d5bb1..f24e83c9e83 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7510,7 +7510,7 @@ def request_remove_replicas( def _add_to_processing( state: SchedulerState, ts: TaskState, recommendations: dict, worker_msgs: dict -) -> WorkerState | None: +) -> None: if state.validate: assert not ts.waiting_on assert not ts.who_has @@ -7535,7 +7535,6 @@ def _add_to_processing( # logger.debug("Send job to worker: %s, %s", worker, key) worker_msgs[ws.address] = [_task_to_msg(state, ts)] - return ws def _remove_from_processing( From 14cc157ff5a4e714344b18985626abc13e0e30e0 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 10:46:39 -0600 Subject: [PATCH 59/98] yaml schema fixes Co-authored-by: crusaderky --- distributed/distributed-schema.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 057d251edb9..4eed80be539 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -119,10 +119,11 @@ properties: worker-saturation: type: number + exclusiveMinimum: 0 description: | Controls how many root tasks are sent to workers (like a `readahead`). - `max(floor(worker-saturation * worker.nthreads), 1)` root tasks are sent to a + Up to worker-saturation * nthreads root tasks are sent to a worker at a time. If `.inf`, all runnable tasks are immediately sent to workers. Allowing oversaturation (> 1.0) means a worker will start running a new root task as From f5d7be443d231634cdd9bc6d3205cff7b6fb757f Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 07:58:10 -0600 Subject: [PATCH 60/98] `topk` -> `peekn` The previous naming and docstring was just wrong. --- distributed/collections.py | 16 ++++++++-------- distributed/scheduler.py | 2 +- distributed/tests/test_collections.py | 14 +++++++------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/distributed/collections.py b/distributed/collections.py index 507ed868139..9d90e477571 100644 --- a/distributed/collections.py +++ b/distributed/collections.py @@ -113,19 +113,19 @@ def pop(self) -> T: self._data.discard(value) return value - def topk(self, k: int) -> Iterator[T]: - "Iterator over the largest K elements. This is O(1) for k == 1, O(n*logn) otherwise." - if k <= 0: + def peekn(self, n: int) -> Iterator[T]: + "Iterator over the N smallest elements. This is O(1) for n == 1, O(n*logn) otherwise." + if n <= 0: return # empty iterator - if k == 1: + if n == 1: yield self.peek() else: - # NOTE: we could pop K items off the queue, then push them back. - # But copying the list K times is probably slower than just sorting it + # NOTE: we could pop N items off the queue, then push them back. + # But copying the list N times is probably slower than just sorting it # with fast C code. # If we had a `heappop` that sliced the list instead of popping from it, - # we could implement an optimized version for small `k`s. - yield from itertools.islice(self.sorted(), k) + # we could implement an optimized version for small `n`s. + yield from itertools.islice(self.sorted(), n) def peekright(self) -> T: """Return one of the largest elements (not necessarily the largest!) without diff --git a/distributed/scheduler.py b/distributed/scheduler.py index f24e83c9e83..d994a5a5815 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2940,7 +2940,7 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): if not math.isinf(self.WORKER_SATURATION) and self.queued: for qts in reversed( list( - self.queued.topk(_task_slots_available(ws, self.WORKER_SATURATION)) + self.queued.peekn(_task_slots_available(ws, self.WORKER_SATURATION)) ) ): if self.validate: diff --git a/distributed/tests/test_collections.py b/distributed/tests/test_collections.py index b1e1da6aa24..89820419145 100644 --- a/distributed/tests/test_collections.py +++ b/distributed/tests/test_collections.py @@ -134,20 +134,20 @@ def test_heapset(): heap.add(cx) assert cx in heap - # Test topk() + # Test peekn() heap.add(cy) heap.add(cw) heap.add(cz) heap.add(cx) - assert list(heap.topk(3)) == [cy, cx, cz] + assert list(heap.peekn(3)) == [cy, cx, cz] heap.remove(cz) - assert list(heap.topk(10)) == [cy, cx, cw] - assert list(heap.topk(0)) == [] - assert list(heap.topk(-1)) == [] + assert list(heap.peekn(10)) == [cy, cx, cw] + assert list(heap.peekn(0)) == [] + assert list(heap.peekn(-1)) == [] heap.remove(cy) - assert list(heap.topk(1)) == [cx] + assert list(heap.peekn(1)) == [cx] heap.remove(cw) - assert list(heap.topk(1)) == [cx] + assert list(heap.peekn(1)) == [cx] # Test resilience to failure in key() bad_key = C("bad_key", 0) From 704b485ceb7de4a33ad1eaa2912cf59581dc795d Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 10:39:11 -0600 Subject: [PATCH 61/98] driveby: fix `saturated` docstring --- distributed/scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index d994a5a5815..e519575fa72 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1258,7 +1258,7 @@ class SchedulerState: * **idle:** ``{WorkerState}``: Set of workers that are currently in running state and not fully utilized * **saturated:** ``{WorkerState}``: - Set of workers that are not over-utilized + Set of workers that are fully utilized. May include non-running workers. * **running:** ``{WorkerState}``: Set of workers that are currently in running state From 38e0598b61eca3da2022cb1ba9e431b4e3635b75 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 10:42:03 -0600 Subject: [PATCH 62/98] driveby: clarify `finish2` name --- distributed/scheduler.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index e519575fa72..f2a79243fa4 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1541,11 +1541,11 @@ def _transition( if not stimulus_id: stimulus_id = STIMULUS_ID_UNSET - finish2 = ts._state + actual_finish = ts._state # FIXME downcast antipattern scheduler = cast(Scheduler, self) scheduler.transition_log.append( - (key, start, finish2, recommendations, stimulus_id, time()) + (key, start, actual_finish, recommendations, stimulus_id, time()) ) if self.validate: if stimulus_id == STIMULUS_ID_UNSET: @@ -1557,7 +1557,7 @@ def _transition( key, start, finish, - ts.state, + actual_finish, dict(recommendations), ) if self.plugins: @@ -1568,7 +1568,7 @@ def _transition( self.tasks[ts.key] = ts for plugin in list(self.plugins.values()): try: - plugin.transition(key, start, finish2, *args, **kwargs) + plugin.transition(key, start, actual_finish, *args, **kwargs) except Exception: logger.info("Plugin failed with exception", exc_info=True) if ts.state == "forgotten": From d47e80dc359b6edcd17eb2686570b6582075f7f2 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 14:54:13 -0600 Subject: [PATCH 63/98] Split up `decide_worker`, remove recs This overhauls `decide_worker` into separate methods for different cases. More importantly, it explicitly turns `transition_waiting_processing` into the primary dispatch mechanism for ready tasks. All ready tasks (deps in memory) now always get recommended to processing, regardless of whether there are any workers in the cluster, whether the have restrictions, whether they're root-ish, etc. `transition_waiting_processing` then decides how to handle them (depending on whether they're root-ish or not), and calls the appropriate `decide_worker` method to search for a worker. If a worker isn't available, then it recommends them off to `queued` or `no-worker` (depending, again, on whether they're root-ish and the WORKER_SATURATION setting). This also updates the `no-worker` state to better match `queued`. Before, `bulk_schedule_after_adding_worker` would send `no-worker` tasks to `waiting`, which would then send them to `processing`. This was weird, because in order to be in `no-worker`, they should already be ready to run (just in need of a worker). So going straight to `processing` makes more sense than sending a ready task back to waiting. Finally, this adds a `SchedulerState.is_rootish` helper. Not quite the static field on a task @fjetter wants in #6922, but a step in that direction. --- distributed/scheduler.py | 375 ++++++++++++++++++++++----------------- 1 file changed, 209 insertions(+), 166 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index f2a79243fa4..afa797dd908 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1677,11 +1677,8 @@ def transition_released_waiting(self, key, stimulus_id): 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" + # NOTE: waiting->processing will send tasks to queued or no-worker as necessary + recommendations[key] = "processing" return recommendations, client_msgs, worker_msgs except Exception as e: @@ -1692,43 +1689,21 @@ def transition_released_waiting(self, key, stimulus_id): pdb.set_trace() raise - def transition_no_worker_waiting(self, key, stimulus_id): + def transition_no_worker_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.actor, f"Actors can't be in `no-worker`: {ts}" 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" + if ws := self.decide_worker_non_rootish(ts): + self.unrunnable.discard(ts) + worker_msgs = _add_to_processing(self, ts, ws) + # If no worker, task just stays in `no-worker` return recommendations, client_msgs, worker_msgs except Exception as e: @@ -1781,100 +1756,135 @@ def transition_no_worker_memory( pdb.set_trace() raise - def decide_worker( - self, ts: TaskState, recommendations: dict[str, str] + def decide_worker_rootish_queuing_disabled( + self, ts: TaskState ) -> WorkerState | None: - """ - Decide on a worker for task *ts*. Return a WorkerState. + """Pick a worker for a runnable root-ish task, without queuing. - If it's a root or root-like task, we place it with its relatives to - reduce future data tansfer. + This attempts to schedule sibling tasks on the same worker, reducing future data + transfer. It does not consider the location of dependencies, since they'll end + up on every worker anyway. - If it has dependencies or restrictions, we use - `decide_worker_from_deps_and_restrictions`. + It assumes it's being called on a batch of tasks in priority order, and + maintains state in `SchedulerState.last_root_worker` and + `SchedulerState.last_root_worker_tasks_left` to achieve this. - Otherwise, we pick the least occupied worker, or pick from all workers - in a round-robin fashion. - """ - if not self.workers: - return None + This will send every runnable task to a worker, often causing root task + overproduction. - tg = ts.group - valid_workers = self.valid_workers(ts) + Returns + ------- + ws: WorkerState | None + The worker to assign the task to. If there are no workers in the cluster, + returns None, in which case the task should be transitioned to + ``no-worker``. + """ + if self.validate: + assert self.is_rootish(ts) + assert math.isinf(self.WORKER_SATURATION) - if ( - valid_workers is not None - and not valid_workers - and not ts.loose_restrictions - ): - recommendations[ts.key] = "no-worker" + pool = self.idle.values() if self.idle else self.running + if not pool: return None - # Group is larger than cluster with few dependencies? - # This is a root task. - # Queue it, or if queuing is disabled, schedule near previous root task - # to 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 + lws = self.last_root_worker + if not ( + lws + and self.last_root_worker_tasks_left + and self.workers.get(lws.address) is lws ): - # Queuing disabled case - if math.isinf(self.WORKER_SATURATION): - pool = self.idle.values() if self.idle else self.running - if not pool: - recommendations[ts.key] = "no-worker" - return None - - lws = self.last_root_worker - if not ( - lws - and self.last_root_worker_tasks_left - and self.workers.get(lws.address) is lws - ): - # Last-used worker is full or unknown; pick a new worker for the next few tasks - ws = self.last_root_worker = min( - pool, key=lambda ws: len(ws.processing) / ws.nthreads - ) - # TODO better batching metric (`len(tg)` is not necessarily the total number of root tasks!) - self.last_root_worker_tasks_left = math.floor( - (len(tg) / self.total_nthreads) * ws.nthreads - ) - else: - ws = lws + # Last-used worker is full or unknown; pick a new worker for the next few tasks + ws = self.last_root_worker = min( + pool, key=lambda ws: len(ws.processing) / ws.nthreads + ) + # TODO better batching metric (`len(tg)` is not necessarily the total number of root tasks!) + self.last_root_worker_tasks_left = math.floor( + (len(ts.group) / self.total_nthreads) * ws.nthreads + ) + else: + ws = lws - self.last_root_worker_tasks_left -= 1 - return ws + self.last_root_worker_tasks_left -= 1 - # Queuing enabled case - if not self.idle: - # All workers busy? Task gets/stays queued. - if self.validate: - assert ts.key not in recommendations, (ts, recommendations[ts.key]) - if ts.state != "queued": - recommendations[ts.key] = "queued" - return None + if self.validate and ws is not None: + assert self.workers.get(ws.address) is ws + assert ws in self.running, (ws, self.running) + + return ws + + def decide_worker_rootish_queuing_enabled( + self, ts: TaskState + ) -> WorkerState | None: + """Pick a worker for a runnable root-ish task, if not all are busy. + + Picks the least-busy worker out of the ``idle`` workers (idle workers have fewer + tasks running than threads, as set by ``distributed.scheduler.worker-saturation``). + It does not consider the location of dependencies, since they'll end up on every + worker anyway. + + If all workers are full, returns None, meaning the task should transition to + ``queued``. The scheduler will wait to send it to a worker until a thread opens + up. This ensures that downstream tasks always run before new root tasks are + started. + + This does not try to schedule sibling tasks on the same worker; in fact, it + usually does the opposite. Even though this increases subsequent data transfer, + it typically reduces overall memory use by eliminating root task overproduction. + + Returns + ------- + ws: WorkerState | None + The worker to assign the task to. If there are no idle workers, returns + None, in which case the task should be transitioned to ``queued``. + + """ + if self.validate: + assert self.is_rootish(ts) + assert not math.isinf(self.WORKER_SATURATION) + + if not self.idle: + # All workers busy? Task gets/stays queued. + return None - # For root tasks, just pick the least busy worker. - # NOTE: this will lead to worst-case scheduling with regards to co-assignment. - ws = min( - self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads + # Just pick the least busy worker. + # NOTE: this will lead to worst-case scheduling with regards to co-assignment. + ws = min(self.idle.values(), key=lambda ws: len(ws.processing) / ws.nthreads) + if self.validate: + assert not _worker_full(ws, self.WORKER_SATURATION), ( + ws, + _task_slots_available(ws, self.WORKER_SATURATION), ) - if self.validate: - assert not _worker_full(ws, self.WORKER_SATURATION), ( - ws, - _task_slots_available(ws, self.WORKER_SATURATION), - ) - assert ws in self.running, (ws, self.running) - return ws + assert ws in self.running, (ws, self.running) + + if self.validate and ws is not None: + assert self.workers.get(ws.address) is ws + assert ws in self.running, (ws, self.running) + + return ws + + def decide_worker_non_rootish(self, ts: TaskState) -> WorkerState | None: + """Pick a worker for a runnable non-root task, considering dependencies and restrictions. + + Out of eligible workers holding dependencies of ``ts``, selects the worker + where, considering worker backlong and data-transfer costs, the task is + estimated to start running the soonest. + + Returns + ------- + ws: WorkerState | None + The worker to assign the task to. If no workers satisfy the restrictions of + ``ts`` or there are no running workers, returns None, in which case the task + should be transitioned to ``no-worker``. + """ + if not self.running: + return None + valid_workers = self.valid_workers(ts) if valid_workers is None and len(self.running) < len(self.workers): if not self.running: - recommendations[ts.key] = "no-worker" return None - # If there were no restrictions, `valid_workers()` + # If there were no restrictions, `valid_workers()` didn't subset by `running`. valid_workers = self.running if ts.dependencies or valid_workers is not None: @@ -1885,6 +1895,12 @@ def decide_worker( partial(self.worker_objective, ts), ) else: + # TODO if `is_rootish` would always return True for tasks without dependencies, + # we could remove all this logic. The rootish assignment logic would behave + # more or less the same as this, maybe without gauranteed round-robin though? + # This path is only reachable when `ts` doesn't have dependencies, but its + # group is also smaller than the cluster. + # Fastpath when there are no related tasks or restrictions worker_pool = self.idle or self.workers wp_vals = worker_pool.values() @@ -1908,20 +1924,36 @@ def decide_worker( ws = wp_vals[self.n_tasks % n_workers] if self.validate and ws is not None: - assert ws.address in self.workers + assert self.workers.get(ws.address) is ws assert ws in self.running, (ws, self.running) return ws def transition_waiting_processing(self, key, stimulus_id): + """Possibly schedule a ready task. This is the primary dispatch for ready tasks. + + If there's no appropriate worker for the task (but the task is otherwise runnable), + it will be recommended to ``no-worker`` or ``queued``. + """ try: ts: TaskState = self.tasks[key] - recommendations: dict = {} - client_msgs: dict = {} - worker_msgs: dict = {} - _add_to_processing(self, ts, recommendations, worker_msgs) - return recommendations, client_msgs, worker_msgs + if self.is_rootish(ts): + # NOTE: having two root-ish methods is temporary. When the feature flag is removed, + # there should only be one, which combines co-assignment and queuing. + # Eventually, special-casing root tasks might be removed entirely, with better heuristics. + if math.isinf(self.WORKER_SATURATION): + if not (ws := self.decide_worker_rootish_queuing_disabled(ts)): + return {ts.key: "no-worker"}, {}, {} + else: + if not (ws := self.decide_worker_rootish_queuing_enabled(ts)): + return {ts.key: "queued"}, {}, {} + else: + if not (ws := self.decide_worker_non_rootish(ts)): + return {ts.key: "no-worker"}, {}, {} + + worker_msgs = _add_to_processing(self, ts, ws) + return {}, {}, worker_msgs except Exception as e: logger.exception(e) if LOG_PDB: @@ -2447,16 +2479,8 @@ def transition_waiting_queued(self, key, stimulus_id): worker_msgs: dict = {} if self.validate: - assert ts not in self.queued assert not self.idle, (ts, self.idle) - # Copied from `transition_waiting_processing` - 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) + _validate_ready(self, ts) ts.state = "queued" self.queued.add(ts) @@ -2478,11 +2502,7 @@ def transition_waiting_no_worker(self, key, stimulus_id): worker_msgs: dict = {} if self.validate: - assert ts not in self.queued - assert not ts.who_has - assert not ts.exception_blame - assert not ts.processing_on - assert ts not in self.unrunnable + _validate_ready(self, ts) ts.state = "no-worker" self.unrunnable.add(ts) @@ -2530,7 +2550,10 @@ def transition_queued_processing(self, key, stimulus_id): assert not ts.actor, f"Actors can't be queued: {ts}" assert ts in self.queued - _add_to_processing(self, ts, recommendations, worker_msgs) + if ws := self.decide_worker_rootish_queuing_enabled(ts): + self.queued.discard(ts) + worker_msgs = _add_to_processing(self, ts, ws) + # If no worker, task just stays `queued` return recommendations, client_msgs, worker_msgs except Exception as e: @@ -2655,7 +2678,7 @@ def transition_released_forgotten(self, key, stimulus_id): ("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", "processing"): transition_no_worker_processing, ("no-worker", "memory"): transition_no_worker_memory, ("released", "forgotten"): transition_released_forgotten, ("memory", "forgotten"): transition_memory_forgotten, @@ -2668,6 +2691,23 @@ def transition_released_forgotten(self, key, stimulus_id): # Assigning Tasks to Workers # ############################## + def is_rootish(self, ts: TaskState) -> bool: + """ + Whether ``ts`` is a root or root-like task. + + Root-ish tasks are part of a group that's larger than the cluster, + and have few or no dependencies. + """ + if ts.resource_restrictions or ts.worker_restrictions or ts.host_restrictions: + return False + tg = ts.group + # TODO short-circuit to True if `not ts.dependencies`? + return ( + len(tg) > self.total_nthreads * 2 + and len(tg.dependencies) < 5 + and sum(map(len, tg.dependencies)) < 5 + ) + def _set_duration_estimate(self, ts: TaskState, ws: WorkerState) -> None: """Estimate task duration using worker state and task state. @@ -2928,14 +2968,13 @@ def _reevaluate_occupancy_worker(self, ws: WorkerState): for ts in ws.processing: steal.recalculate_cost(ts) - def bulk_schedule_after_adding_worker(self, ws: WorkerState): + def bulk_schedule_after_adding_worker(self, ws: WorkerState) -> dict[str, str]: """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. """ - recommendations: dict[str, str] = {} - + maybe_runnable: list[TaskState] = [] # Schedule any queued tasks onto the new worker if not math.isinf(self.WORKER_SATURATION) and self.queued: for qts in reversed( @@ -2948,20 +2987,18 @@ def bulk_schedule_after_adding_worker(self, ws: WorkerState): assert not qts.processing_on assert not qts.waiting_on - recommendations[qts.key] = "processing" + maybe_runnable.append(qts) - now_runnable: list[TaskState] = [] + # Schedule any restricted tasks onto the new worker, if the worker can run them for ts in self.unrunnable: valid = self.valid_workers(ts) if valid is None or ws in valid: - now_runnable.append(ts) + maybe_runnable.append(ts) + # Recommendations are processed LIFO, hence the reversed order - now_runnable.sort(key=operator.attrgetter("priority"), reverse=True) - for ts in now_runnable: - # TODO this assumes unrunnable tasks are always higher-priority than queued - # tasks. Should we sort all of the recommendations instead? - recommendations[ts.key] = "waiting" - return recommendations + maybe_runnable.sort(key=operator.attrgetter("priority"), reverse=True) + # Note not all will necessarily be run; transition->processing will decide + return {ts.key: "processing" for ts in maybe_runnable} class Scheduler(SchedulerState, ServerNode): @@ -7508,33 +7545,39 @@ def request_remove_replicas( ) +def _validate_ready(state: SchedulerState, ts: TaskState): + "Validation for ready states (processing, queued, no-worker)" + 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 state.unrunnable + assert ts not in state.queued + assert all(dts.who_has for dts in ts.dependencies) + + def _add_to_processing( - state: SchedulerState, ts: TaskState, recommendations: dict, worker_msgs: dict -) -> None: + state: SchedulerState, ts: TaskState, ws: WorkerState +) -> dict[str, list]: + "Set a task as processing on a worker, and return the worker messages to send." if state.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 state.unrunnable - assert all(dts.who_has for dts in ts.dependencies) - - if ws := state.decide_worker(ts, recommendations): - state._set_duration_estimate(ts, ws) - ts.processing_on = ws - ts.state = "processing" - state.queued.discard(ts) - state.acquire_resources(ts, ws) - state.check_idle_saturated(ws) - state.n_tasks += 1 - - if ts.actor: - ws.actors.add(ts) + _validate_ready(state, ts) + assert ts not in ws.processing + assert ws in state.running, state.running + assert (o := state.workers.get(ws.address)) is ws, (ws, o) + + state._set_duration_estimate(ts, ws) + ts.processing_on = ws + ts.state = "processing" + state.acquire_resources(ts, ws) + state.check_idle_saturated(ws) + state.n_tasks += 1 - # logger.debug("Send job to worker: %s, %s", worker, key) + if ts.actor: + ws.actors.add(ts) - worker_msgs[ws.address] = [_task_to_msg(state, ts)] + return {ws.address: [_task_to_msg(state, ts)]} def _remove_from_processing( From 2cc86318ad33dc1e3d11d7a7e01bc600c35c45f7 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 16:49:37 -0600 Subject: [PATCH 64/98] remove no_worker->memory just to see what happens The only valid way I can imagine any of these happening is `client.scatter` within a worker. If this is actually needed, I guess I should add an equivalent for queued? --- distributed/scheduler.py | 85 ++++++++++++++++++++-------------------- 1 file changed, 43 insertions(+), 42 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index afa797dd908..e74a033801c 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1714,47 +1714,48 @@ def transition_no_worker_processing(self, key, stimulus_id): 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 + # FIXME This almost certainly shouldn't happen + # 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_rootish_queuing_disabled( self, ts: TaskState @@ -2679,7 +2680,7 @@ def transition_released_forgotten(self, key, stimulus_id): ("processing", "erred"): transition_processing_erred, ("no-worker", "released"): transition_no_worker_released, ("no-worker", "processing"): transition_no_worker_processing, - ("no-worker", "memory"): transition_no_worker_memory, + # ("no-worker", "memory"): transition_no_worker_memory, ("released", "forgotten"): transition_released_forgotten, ("memory", "forgotten"): transition_memory_forgotten, ("erred", "released"): transition_erred_released, From 100118ad89b09762677165ca8e2627157d4afa75 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 16:24:31 -0600 Subject: [PATCH 65/98] Render scheduler state graph with inline graphviz --- .readthedocs.yaml | 2 + docs/requirements.txt | 1 + docs/source/conf.py | 1 + docs/source/images/task-state.dot | 19 ----- docs/source/images/task-state.svg | 113 ------------------------------ docs/source/scheduling-state.rst | 22 +++++- 6 files changed, 24 insertions(+), 134 deletions(-) delete mode 100644 docs/source/images/task-state.dot delete mode 100644 docs/source/images/task-state.svg diff --git a/.readthedocs.yaml b/.readthedocs.yaml index bb4bcb544e5..a8312771ddd 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -4,6 +4,8 @@ build: os: ubuntu-20.04 tools: python: "3.9" + apt_packages: + - dot sphinx: configuration: docs/source/conf.py diff --git a/docs/requirements.txt b/docs/requirements.txt index 51c002f1a80..0896f243a8c 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -6,3 +6,4 @@ git+https://github.com/dask/dask sphinx dask-sphinx-theme>=3.0.0 sphinx-click +graphviz diff --git a/docs/source/conf.py b/docs/source/conf.py index e048e4f7747..ce72b33a8a4 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -44,6 +44,7 @@ "sphinx.ext.autosummary", "sphinx.ext.extlinks", "sphinx.ext.intersphinx", + "sphinx.ext.graphviz", "numpydoc", "sphinx_click.ext", ] diff --git a/docs/source/images/task-state.dot b/docs/source/images/task-state.dot deleted file mode 100644 index 52ec18ff570..00000000000 --- a/docs/source/images/task-state.dot +++ /dev/null @@ -1,19 +0,0 @@ -digraph{ - graph [ - bgcolor="#FFFFFFF00", - rankdir=LR, - ]; - released1 [label=released]; - released2 [label=released]; - new -> released1; - released1 -> waiting; - waiting -> processing; - waiting -> "no-worker"; - "no-worker" -> waiting; - "no-worker" -> processing; - processing -> memory; - processing -> error; - error -> forgotten; - memory -> released2; - released2 -> forgotten; -} diff --git a/docs/source/images/task-state.svg b/docs/source/images/task-state.svg deleted file mode 100644 index 49e6e81c98d..00000000000 --- a/docs/source/images/task-state.svg +++ /dev/null @@ -1,113 +0,0 @@ - - - - - - -%3 - - -released1 - -released - - -waiting - -waiting - - -released1->waiting - - - - -released2 - -released - - -forgotten - -forgotten - - -released2->forgotten - - - - -new - -new - - -new->released1 - - - - -processing - -processing - - -waiting->processing - - - - -no-worker - -no-worker - - -waiting->no-worker - - - - -memory - -memory - - -processing->memory - - - - -error - -error - - -processing->error - - - - -no-worker->waiting - - - - -no-worker->processing - - - - -memory->released2 - - - - -error->forgotten - - - - - diff --git a/docs/source/scheduling-state.rst b/docs/source/scheduling-state.rst index 3e8dfb4e81d..88e52c519c4 100644 --- a/docs/source/scheduling-state.rst +++ b/docs/source/scheduling-state.rst @@ -57,8 +57,26 @@ notably ``released``, ``waiting``, ``no-worker``, ``processing``, Tasks flow along the following states with the following allowed transitions: -.. image:: images/task-state.svg - :alt: Dask scheduler task states +.. digraph:: scheduler_task_states + :alt: Dask scheduler task states + + graph [ + bgcolor="#FFFFFFF00", + rankdir=LR, + ]; + released1 [label=released]; + released2 [label=released]; + new -> released1; + released1 -> waiting; + waiting -> processing; + waiting -> "no-worker"; + "no-worker" -> waiting; + "no-worker" -> processing; + processing -> memory; + processing -> error; + error -> forgotten; + memory -> released2; + released2 -> forgotten; * *Released*: Known but not actively computing or in memory * *Waiting*: On track to be computed, waiting on dependencies to arrive in From 842ee7185912cce663bbfc32fcb624c4c01e9c29 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 16:42:40 -0600 Subject: [PATCH 66/98] Update scheduling state graph and docs slightly --- docs/source/scheduling-state.rst | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/source/scheduling-state.rst b/docs/source/scheduling-state.rst index 88e52c519c4..045c393b194 100644 --- a/docs/source/scheduling-state.rst +++ b/docs/source/scheduling-state.rst @@ -52,7 +52,7 @@ Task State ---------- Internally, the scheduler moves tasks between a fixed set of states, -notably ``released``, ``waiting``, ``no-worker``, ``processing``, +notably ``released``, ``waiting``, ``no-worker``, ``queued``, ``processing``, ``memory``, ``error``. Tasks flow along the following states with the following allowed transitions: @@ -70,8 +70,12 @@ Tasks flow along the following states with the following allowed transitions: released1 -> waiting; waiting -> processing; waiting -> "no-worker"; - "no-worker" -> waiting; "no-worker" -> processing; + "no-worker" -> released2; + waiting -> queued; + queued -> processing; + queued -> released2; + processing -> released2; processing -> memory; processing -> error; error -> forgotten; @@ -84,6 +88,7 @@ Tasks flow along the following states with the following allowed transitions: * *No-worker*: Ready to be computed, but no appropriate worker exists (for example because of resource restrictions, or because no worker is connected at all). +* *Queued*: Ready to be computed, but all workers are already full * *Processing*: All dependencies are available and the task is assigned to a worker for compute (the scheduler doesn't know whether it's in a worker queue or actively being computed). From dd88b0d678d55616c3890cd727cdfab9d067ed99 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 17:47:10 -0600 Subject: [PATCH 67/98] apt package is graphviz, not dot --- .readthedocs.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.readthedocs.yaml b/.readthedocs.yaml index a8312771ddd..ea4db2acf9a 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -5,7 +5,7 @@ build: tools: python: "3.9" apt_packages: - - dot + - graphviz sphinx: configuration: docs/source/conf.py From e17c624264e72db9f4fa4ed835335a26d8540e42 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 24 Aug 2022 17:53:20 -0600 Subject: [PATCH 68/98] Revert "remove no_worker->memory just to see what happens" This reverts commit 2cc86318ad33dc1e3d11d7a7e01bc600c35c45f7. --- distributed/scheduler.py | 85 ++++++++++++++++++++-------------------- 1 file changed, 42 insertions(+), 43 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index d6b434dd749..b90139be30e 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1714,48 +1714,47 @@ def transition_no_worker_processing(self, key, stimulus_id): pdb.set_trace() raise - # FIXME This almost certainly shouldn't happen - # 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 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_rootish_queuing_disabled( self, ts: TaskState @@ -2705,7 +2704,7 @@ def transition_released_forgotten(self, key, stimulus_id): ("processing", "erred"): transition_processing_erred, ("no-worker", "released"): transition_no_worker_released, ("no-worker", "processing"): transition_no_worker_processing, - # ("no-worker", "memory"): transition_no_worker_memory, + ("no-worker", "memory"): transition_no_worker_memory, ("released", "forgotten"): transition_released_forgotten, ("memory", "forgotten"): transition_memory_forgotten, ("erred", "released"): transition_erred_released, From 06d60fe836176a6fa4d7821b0f52e19f14da8245 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 13:03:48 -0600 Subject: [PATCH 69/98] Revert "Render scheduler state graph with inline graphviz" This reverts commits dd88b0d678d55616c3890cd727cdfab9d067ed99 842ee7185912cce663bbfc32fcb624c4c01e9c29 100118ad89b09762677165ca8e2627157d4afa75 --- .readthedocs.yaml | 2 - docs/requirements.txt | 1 - docs/source/conf.py | 1 - docs/source/images/task-state.dot | 19 +++++ docs/source/images/task-state.svg | 113 ++++++++++++++++++++++++++++++ docs/source/scheduling-state.rst | 29 +------- 6 files changed, 135 insertions(+), 30 deletions(-) create mode 100644 docs/source/images/task-state.dot create mode 100644 docs/source/images/task-state.svg diff --git a/.readthedocs.yaml b/.readthedocs.yaml index ea4db2acf9a..bb4bcb544e5 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -4,8 +4,6 @@ build: os: ubuntu-20.04 tools: python: "3.9" - apt_packages: - - graphviz sphinx: configuration: docs/source/conf.py diff --git a/docs/requirements.txt b/docs/requirements.txt index 0896f243a8c..51c002f1a80 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -6,4 +6,3 @@ git+https://github.com/dask/dask sphinx dask-sphinx-theme>=3.0.0 sphinx-click -graphviz diff --git a/docs/source/conf.py b/docs/source/conf.py index ce72b33a8a4..e048e4f7747 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -44,7 +44,6 @@ "sphinx.ext.autosummary", "sphinx.ext.extlinks", "sphinx.ext.intersphinx", - "sphinx.ext.graphviz", "numpydoc", "sphinx_click.ext", ] diff --git a/docs/source/images/task-state.dot b/docs/source/images/task-state.dot new file mode 100644 index 00000000000..52ec18ff570 --- /dev/null +++ b/docs/source/images/task-state.dot @@ -0,0 +1,19 @@ +digraph{ + graph [ + bgcolor="#FFFFFFF00", + rankdir=LR, + ]; + released1 [label=released]; + released2 [label=released]; + new -> released1; + released1 -> waiting; + waiting -> processing; + waiting -> "no-worker"; + "no-worker" -> waiting; + "no-worker" -> processing; + processing -> memory; + processing -> error; + error -> forgotten; + memory -> released2; + released2 -> forgotten; +} diff --git a/docs/source/images/task-state.svg b/docs/source/images/task-state.svg new file mode 100644 index 00000000000..49e6e81c98d --- /dev/null +++ b/docs/source/images/task-state.svg @@ -0,0 +1,113 @@ + + + + + + +%3 + + +released1 + +released + + +waiting + +waiting + + +released1->waiting + + + + +released2 + +released + + +forgotten + +forgotten + + +released2->forgotten + + + + +new + +new + + +new->released1 + + + + +processing + +processing + + +waiting->processing + + + + +no-worker + +no-worker + + +waiting->no-worker + + + + +memory + +memory + + +processing->memory + + + + +error + +error + + +processing->error + + + + +no-worker->waiting + + + + +no-worker->processing + + + + +memory->released2 + + + + +error->forgotten + + + + + diff --git a/docs/source/scheduling-state.rst b/docs/source/scheduling-state.rst index 045c393b194..3e8dfb4e81d 100644 --- a/docs/source/scheduling-state.rst +++ b/docs/source/scheduling-state.rst @@ -52,35 +52,13 @@ Task State ---------- Internally, the scheduler moves tasks between a fixed set of states, -notably ``released``, ``waiting``, ``no-worker``, ``queued``, ``processing``, +notably ``released``, ``waiting``, ``no-worker``, ``processing``, ``memory``, ``error``. Tasks flow along the following states with the following allowed transitions: -.. digraph:: scheduler_task_states - :alt: Dask scheduler task states - - graph [ - bgcolor="#FFFFFFF00", - rankdir=LR, - ]; - released1 [label=released]; - released2 [label=released]; - new -> released1; - released1 -> waiting; - waiting -> processing; - waiting -> "no-worker"; - "no-worker" -> processing; - "no-worker" -> released2; - waiting -> queued; - queued -> processing; - queued -> released2; - processing -> released2; - processing -> memory; - processing -> error; - error -> forgotten; - memory -> released2; - released2 -> forgotten; +.. image:: images/task-state.svg + :alt: Dask scheduler task states * *Released*: Known but not actively computing or in memory * *Waiting*: On track to be computed, waiting on dependencies to arrive in @@ -88,7 +66,6 @@ Tasks flow along the following states with the following allowed transitions: * *No-worker*: Ready to be computed, but no appropriate worker exists (for example because of resource restrictions, or because no worker is connected at all). -* *Queued*: Ready to be computed, but all workers are already full * *Processing*: All dependencies are available and the task is assigned to a worker for compute (the scheduler doesn't know whether it's in a worker queue or actively being computed). From 96d59eb2275f18d0a3ff6b1a256fccf3a802fd78 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 13:07:27 -0600 Subject: [PATCH 70/98] Update scheduling state docs --- docs/source/images/task-state.dot | 6 +- docs/source/images/task-state.svg | 185 +++++++++++++++++++----------- docs/source/scheduling-state.rst | 3 +- 3 files changed, 124 insertions(+), 70 deletions(-) diff --git a/docs/source/images/task-state.dot b/docs/source/images/task-state.dot index 52ec18ff570..89536807c7d 100644 --- a/docs/source/images/task-state.dot +++ b/docs/source/images/task-state.dot @@ -9,8 +9,12 @@ digraph{ released1 -> waiting; waiting -> processing; waiting -> "no-worker"; - "no-worker" -> waiting; "no-worker" -> processing; + "no-worker" -> released2; + waiting -> queued; + queued -> processing; + queued -> released2; + processing -> released2; processing -> memory; processing -> error; error -> forgotten; diff --git a/docs/source/images/task-state.svg b/docs/source/images/task-state.svg index 49e6e81c98d..8e306574ab7 100644 --- a/docs/source/images/task-state.svg +++ b/docs/source/images/task-state.svg @@ -1,113 +1,162 @@ - - - - -%3 - + + + + -released1 - -released + +released1 + +released -waiting - -waiting + +waiting + +waiting -released1->waiting - - + +released1->waiting + + -released2 - -released + +released2 + +released -forgotten - -forgotten + +forgotten + +forgotten -released2->forgotten - - + +released2->forgotten + + -new - -new + +new + +new -new->released1 - - + +new->released1 + + -processing - -processing + +processing + +processing -waiting->processing - - + +waiting->processing + + -no-worker - -no-worker + +no-worker + +no-worker -waiting->no-worker - - + +waiting->no-worker + + + + + +queued + +queued + + + +waiting->queued + + + + + +processing->released2 + + -memory - -memory + +memory + +memory -processing->memory - - + +processing->memory + + -error - -error + +error + +error -processing->error - - + +processing->error + + - -no-worker->waiting - - + + +no-worker->released2 + + -no-worker->processing - - + +no-worker->processing + + + + + +queued->released2 + + + + + +queued->processing + + -memory->released2 - - + +memory->released2 + + -error->forgotten - - + +error->forgotten + + diff --git a/docs/source/scheduling-state.rst b/docs/source/scheduling-state.rst index 3e8dfb4e81d..0560242b9bd 100644 --- a/docs/source/scheduling-state.rst +++ b/docs/source/scheduling-state.rst @@ -52,7 +52,7 @@ Task State ---------- Internally, the scheduler moves tasks between a fixed set of states, -notably ``released``, ``waiting``, ``no-worker``, ``processing``, +notably ``released``, ``waiting``, ``no-worker``, ``queued``, ``processing``, ``memory``, ``error``. Tasks flow along the following states with the following allowed transitions: @@ -66,6 +66,7 @@ Tasks flow along the following states with the following allowed transitions: * *No-worker*: Ready to be computed, but no appropriate worker exists (for example because of resource restrictions, or because no worker is connected at all). +* *Queued*: Ready to be computed, but all workers are already full * *Processing*: All dependencies are available and the task is assigned to a worker for compute (the scheduler doesn't know whether it's in a worker queue or actively being computed). From 3240a431e9e44e3d29eb72aa6bd7c648e57f59e1 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 13:27:46 -0600 Subject: [PATCH 71/98] `test_root_task_overproduction` adaptive data size Still maybe not a test that should run in CI, I just like how real-world it is. Let's see if picking the task size based on available memory helps on windows. --- distributed/tests/test_scheduler.py | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 27645768824..d4b6b327261 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -22,7 +22,15 @@ import dask from dask import delayed -from dask.utils import apply, parse_bytes, parse_timedelta, stringify, tmpfile, typename +from dask.utils import ( + apply, + format_bytes, + parse_bytes, + parse_timedelta, + stringify, + tmpfile, + typename, +) from distributed import ( CancelledError, @@ -288,9 +296,17 @@ async def test_root_task_overproduction(c, s, *nannies): def big_data(size: int) -> str: return "x" * size - roots = [ - big_data(parse_bytes("300 MiB"), dask_key_name=f"root-{i}") for i in range(16) - ] + available = await c.run( + lambda dask_worker: dask_worker.memory_manager.memory_limit + - dask_worker.monitor.get_process_memory() + ) + print({k: format_bytes(v) for k, v in available.items()}) + min_available = min(available.values()) + # leave room so 2 can comfortably fit, but 3 can't + task_size = round(min_available / 2.9) + print(f"task size: {format_bytes(task_size)}") + + roots = [big_data(task_size, dask_key_name=f"root-{i}") for i in range(16)] passthrough = [delayed(slowidentity)(x) for x in roots] memory_consumed = [delayed(len)(x) for x in passthrough] reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] From 9344dd92fa8c4add4d344430fb469ae223155192 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 14:06:52 -0600 Subject: [PATCH 72/98] Don't assert rootish in `decide_worker` --- distributed/scheduler.py | 7 +++++-- distributed/tests/test_scheduler.py | 19 +++++++++++++++++-- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index b90139be30e..9b1358562e4 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1780,7 +1780,7 @@ def decide_worker_rootish_queuing_disabled( ``no-worker``. """ if self.validate: - assert self.is_rootish(ts) + # See root-ish-ness note below in `decide_worker_rootish_queuing_enabled` assert math.isinf(self.WORKER_SATURATION) pool = self.idle.values() if self.idle else self.running @@ -1839,7 +1839,10 @@ def decide_worker_rootish_queuing_enabled( """ if self.validate: - assert self.is_rootish(ts) + # We don't `assert self.is_rootish(ts)` here, because that check is dependent on + # cluster size. It's possible a task looked root-ish when it was queued, but the + # cluster has since scaled up and it no longer does when coming out of the queue. + # If `is_rootish` changes to a static definition, then add that assertion here. assert not math.isinf(self.WORKER_SATURATION) if not self.idle: diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index d4b6b327261..4564ee773bb 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -512,17 +512,32 @@ async def test_no_valid_workers_loose_restrictions(client, s, a, b, c): assert result == 2 +@pytest.mark.parametrize("queue", [False, True]) @gen_cluster(client=True, nthreads=[]) -async def test_no_workers(client, s): +async def test_no_workers(client, s, queue): + if queue: + s.WORKER_SATURATION = 1.0 + else: + s.WORKER_SATURATION = float("inf") + x = client.submit(inc, 1) while not s.tasks: await asyncio.sleep(0.01) - assert s.tasks[x.key] in s.unrunnable + ts = s.tasks[x.key] + if queue: + assert ts in s.queued + assert ts.state == "queued" + else: + assert ts in s.unrunnable + assert ts.state == "no-worker" with pytest.raises(TimeoutError): await asyncio.wait_for(x, 0.05) + async with Worker(s.address, nthreads=1): + await wait(x) + @gen_cluster(nthreads=[]) async def test_retire_workers_empty(s): From aa8e1dbecc1112f7a272d09e535c9cea8ad61b7f Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 19:08:25 -0600 Subject: [PATCH 73/98] improve `test_queued_paused` --- distributed/tests/test_scheduler.py | 46 +++++++++++++++++------------ 1 file changed, 27 insertions(+), 19 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 4564ee773bb..73ede3030af 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -25,7 +25,6 @@ from dask.utils import ( apply, format_bytes, - parse_bytes, parse_timedelta, stringify, tmpfile, @@ -318,46 +317,55 @@ def big_data(size: int) -> str: assert pids == [n.pid for n in nannies] -@pytest.mark.parametrize("withhold", [True, False]) +@pytest.mark.parametrize("queue", [True, False]) @gen_cluster( client=True, nthreads=[("", 2)] * 2, - worker_kwargs={"memory_limit": "1.0 GiB"}, - Worker=Nanny, config={ - "distributed.scheduler.worker-saturation": 1.0, - # With typical overhead, 1 task can be in memory but the second will trigger a pause - "distributed.worker.memory.pause": 0.4, + "distributed.worker.memory.pause": False, "distributed.worker.memory.target": False, "distributed.worker.memory.spill": False, "distributed.scheduler.work-stealing": False, }, ) -async def test_queued_paused_released(c, s, a, b, withhold): - if not withhold: +async def test_queued_paused_new_worker(c, s, a, b, queue): + if queue: + s.WORKER_SATURATION = 1.0 + else: s.WORKER_SATURATION = float("inf") - @delayed(pure=True) # type: ignore - def big_data(size: int) -> str: - return "x" * size + f1s = c.map(slowinc, range(16)) + f2s = c.map(slowinc, f1s) + final = c.submit(sum, *f2s) + del f1s, f2s - roots = [ - big_data(parse_bytes("200 MiB"), dask_key_name=f"root-{i}") for i in range(16) - ] - memory_consumed = [delayed(len)(x) for x in roots] - f = c.compute(sum(memory_consumed)) + while not a.data or not b.data: + await asyncio.sleep(0.01) + + # manually pause the workers + a.status = Status.paused + b.status = Status.paused - while s.running: # wait for both workers pausing to hit the scheduler + while a.state.executing_count or b.state.executing_count: + # wait for workers to stop await asyncio.sleep(0.01) + while s.running: + # wait for workers pausing to hit the scheduler + await asyncio.sleep(0.01) + + assert not s.idle + assert not s.running + async with Worker(s.address, nthreads=2) as w: # Tasks are successfully scheduled onto a new worker while not w.state.data: await asyncio.sleep(0.01) - f.release() + del final while s.tasks: await asyncio.sleep(0.01) + assert not s.queued @pytest.mark.parametrize( From ee1a7542e345e0771f4ed61895eadb498f8d830b Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 19:09:40 -0600 Subject: [PATCH 74/98] `test_queued_paused_unpaused` --- distributed/tests/test_scheduler.py | 51 ++++++++++++++++++++++++++--- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 73ede3030af..50c1394719a 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -346,10 +346,6 @@ async def test_queued_paused_new_worker(c, s, a, b, queue): a.status = Status.paused b.status = Status.paused - while a.state.executing_count or b.state.executing_count: - # wait for workers to stop - await asyncio.sleep(0.01) - while s.running: # wait for workers pausing to hit the scheduler await asyncio.sleep(0.01) @@ -368,6 +364,53 @@ async def test_queued_paused_new_worker(c, s, a, b, queue): assert not s.queued +@pytest.mark.parametrize("queue", [True, False]) +@gen_cluster( + client=True, + nthreads=[("", 2)] * 2, + config={ + "distributed.worker.memory.pause": False, + "distributed.worker.memory.target": False, + "distributed.worker.memory.spill": False, + "distributed.scheduler.work-stealing": False, + }, +) +async def test_queued_paused_unpaused(c, s, a, b, queue): + if queue: + s.WORKER_SATURATION = 1.0 + else: + s.WORKER_SATURATION = float("inf") + + f1s = c.map(slowinc, range(16)) + f2s = c.map(slowinc, f1s) + final = c.submit(sum, *f2s) + del f1s, f2s + + while not a.data or not b.data: + await asyncio.sleep(0.01) + + # manually pause the workers + a.status = Status.paused + b.status = Status.paused + + while s.running: + # wait for workers pausing to hit the scheduler + await asyncio.sleep(0.01) + + assert not s.running + assert not s.idle + + # un-pause + a.status = Status.running + b.status = Status.running + while not s.running: + await asyncio.sleep(0.01) + + assert not s.idle # workers should have been (or already were) filled + + await wait(final) + + @pytest.mark.parametrize( "saturation, expected_task_counts", [ From f36a6acc27cac68a26bdfc5982df821fac136354 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 19:10:22 -0600 Subject: [PATCH 75/98] `bulk_schedule_after_adding_worker` docstring --- distributed/scheduler.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 9b1358562e4..0f8bb3136e8 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -3007,10 +3007,9 @@ def _reevaluate_occupancy_worker(self, ws: WorkerState): steal.recalculate_cost(ts) def bulk_schedule_after_adding_worker(self, ws: WorkerState) -> dict[str, str]: - """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. + """Send ``queued`` or ``no-worker`` tasks to ``processing`` that this worker can handle. + + Returns priority-ordered recommendations. """ maybe_runnable: list[TaskState] = [] # Schedule any queued tasks onto the new worker From 78353e178542759cd5dd9c2e6d07c1fb6868170a Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 19:53:50 -0600 Subject: [PATCH 76/98] `test_queued_remove_add_worker` --- distributed/tests/test_scheduler.py | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 50c1394719a..a5641af085a 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -51,6 +51,7 @@ from distributed.utils import TimeoutError from distributed.utils_test import ( BrokenComm, + async_wait_for, captured_logger, cluster, dec, @@ -411,6 +412,31 @@ async def test_queued_paused_unpaused(c, s, a, b, queue): await wait(final) +@gen_cluster( + client=True, + nthreads=[("", 2)] * 2, + config={"distributed.scheduler.worker-saturation": 1.0}, +) +async def test_queued_remove_add_worker(c, s, a, b): + event = Event() + fs = c.map(lambda i: event.wait(), range(10)) + + await async_wait_for( + lambda: len(s.queued) == 6, + timeout=5, + fail_func=lambda: print(list(s.queued.sorted()), s.tasks), + ) + await s.remove_worker(a.address, stimulus_id="fake") + assert len(s.queued) == 8 + + # Add a new worker + async with Worker(s.address, nthreads=2) as w: + await async_wait_for(lambda: len(s.queued) == 6, timeout=5) + + await event.set() + await wait(fs) + + @pytest.mark.parametrize( "saturation, expected_task_counts", [ From 18b7bb5d1f17da3151fe600a471e158d18fb25b7 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 19:55:02 -0600 Subject: [PATCH 77/98] reorder and quicken `test_queued_tasks_rebalance` --- distributed/tests/test_scheduler.py | 96 ++++++++++++++--------------- 1 file changed, 48 insertions(+), 48 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index a5641af085a..71c5f7b784b 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -437,6 +437,54 @@ async def test_queued_remove_add_worker(c, s, a, b): await wait(fs) +@pytest.mark.slow +@gen_cluster( + client=True, + nthreads=[("", 2)] * 2, + config={"distributed.scheduler.worker-saturation": 1.0}, +) +async def test_queued_tasks_rebalance(client, s, a, b): + """ + Test that all queued tasks complete as workers come and go. + + Does not test how well balanced the load was. + """ + roots1 = [delayed(slowinc)(i) for i in range(40)] + roots2 = [delayed(slowinc)(i, delay=0.01) for i in range(len(roots1))] + combined = [x + y for x, y in zip(roots1, roots2)] + agg = [sum(xs) for xs in partition(4, combined)] + + fs = client.compute(agg) + + while len(a.data) + len(b.data) < len(agg) * 0.25: + await asyncio.sleep(0.01) + + # Add a new worker + async with Worker(s.address, nthreads=2) as c: + while not c.data: + await asyncio.sleep(0.01) + + # Now add another + async with Worker(s.address, nthreads=1) as d: + while not d.data: + await asyncio.sleep(0.01) + + # Remove an existing worker + await a.close() + + while len(b.data) + len(c.data) + len(d.data) < len(agg) * 0.75: + await asyncio.sleep(0.01) + + # And a new one + await d.close() + + await client.gather(fs) + assert a.state.tasks + assert b.state.tasks + assert c.state.tasks + assert d.state.tasks + + @pytest.mark.parametrize( "saturation, expected_task_counts", [ @@ -502,54 +550,6 @@ async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor assert len(b.state.tasks) == 9 -@pytest.mark.slow -@gen_cluster( - client=True, - nthreads=[("", 2)] * 2, - config={"distributed.scheduler.worker-saturation": 1.0}, -) -async def test_queued_tasks_rebalance(client, s, a, b): - """ - Test that all queued tasks complete as workers come and go. - - Does not test how well balanced the load was. - """ - roots1 = [delayed(slowinc)(i) for i in range(400)] - roots2 = [delayed(slowinc)(i, delay=0.01) for i in range(len(roots1))] - combined = [x + y for x, y in zip(roots1, roots2)] - agg = [sum(xs) for xs in partition(4, combined)] - - fs = client.compute(agg) - - while len(a.data) + len(b.data) < len(agg) * 0.25: - await asyncio.sleep(0.01) - - # Add a new worker - async with Worker(s.address, nthreads=2) as c: - while not c.data: - await asyncio.sleep(0.01) - - # Now add another - async with Worker(s.address, nthreads=1) as d: - while not d.data: - await asyncio.sleep(0.01) - - # Remove an existing worker - await a.close() - - while len(b.data) + len(c.data) + len(d.data) < len(agg) * 0.75: - await asyncio.sleep(0.01) - - # And a new one - await d.close() - - await client.gather(fs) - assert a.state.tasks - assert b.state.tasks - assert c.state.tasks - assert d.state.tasks - - @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) async def test_move_data_over_break_restrictions(client, s, a, b, c): [x] = await client.scatter([1], workers=b.address) From f3a66dfbd843a4e27ef2ccd2e1d14c0f704271a1 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 20:29:18 -0600 Subject: [PATCH 78/98] queued tasks cause scale-up from empty --- distributed/scheduler.py | 2 +- distributed/tests/test_scheduler.py | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 0f8bb3136e8..816f1d9820f 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7512,7 +7512,7 @@ def adaptive_target(self, target_duration=None): else: cpu = min(tasks_ready, cpu) - if self.unrunnable and not self.workers: + if (self.unrunnable or self.queued) and not self.workers: cpu = max(1, cpu) # add more workers if more than 60% of memory is used diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 71c5f7b784b..203f9d6cdad 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2354,6 +2354,25 @@ async def test_adaptive_target(c, s, a, b): assert s.adaptive_target(target_duration=".1s") == 0 +@pytest.mark.parametrize("queue", [True, False]) +@gen_cluster( + client=True, + nthreads=[], + config={"distributed.scheduler.default-task-durations": {"inc": 1}}, +) +async def test_adaptive_target_empty_cluster(c, s, queue): + if queue: + s.WORKER_SATURATION = 1.0 + else: + s.WORKER_SATURATION = float("inf") + + assert s.adaptive_target() == 0 + + f = c.submit(inc, -1) + await async_wait_for(lambda: s.tasks, timeout=5) + assert s.adaptive_target() == 1 + + @gen_test() async def test_async_context_manager(): async with Scheduler(dashboard_address=":0") as s: From c5f27460df769edc34dd0aa904f136443a24b5a8 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 20:49:01 -0600 Subject: [PATCH 79/98] Improve adaptive target for queued tasks --- distributed/scheduler.py | 11 ++--------- distributed/tests/test_scheduler.py | 8 +++++++- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 816f1d9820f..2e2be1505fd 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7489,15 +7489,8 @@ def adaptive_target(self, target_duration=None): # CPU - # FIXME maintain a proper estimate of queued occupancy!! - # This is merely a hack intended to make queuing sorta work with adaptive scaling - # so people can try it out in the short term (at least the cluster should scale up - # when tasks are queued). - avg_duration = ( - (self.total_occupancy / self.total_nthreads) if self.total_nthreads else 0 - ) - queued_occupancy = len(self.queued) * avg_duration - + # TODO consider any user-specified default task durations for queued tasks + queued_occupancy = len(self.queued) * self.UNKNOWN_TASK_DURATION cpu = math.ceil( (self.total_occupancy + queued_occupancy) / target_duration ) # TODO: threads per worker diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 203f9d6cdad..d97af00a44a 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -2358,7 +2358,6 @@ async def test_adaptive_target(c, s, a, b): @gen_cluster( client=True, nthreads=[], - config={"distributed.scheduler.default-task-durations": {"inc": 1}}, ) async def test_adaptive_target_empty_cluster(c, s, queue): if queue: @@ -2371,6 +2370,13 @@ async def test_adaptive_target_empty_cluster(c, s, queue): f = c.submit(inc, -1) await async_wait_for(lambda: s.tasks, timeout=5) assert s.adaptive_target() == 1 + del f + + if queue: + # only queuing supports fast scale-up for empty clusters https://github.com/dask/distributed/issues/6962 + fs = c.map(inc, range(100)) + await async_wait_for(lambda: len(s.tasks) == len(fs), timeout=5) + assert s.adaptive_target() > 1 @gen_test() From 4b2a209c79e447b3abaae1ef93acebf27dec2521 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 21:00:43 -0600 Subject: [PATCH 80/98] don't need that fail_func --- distributed/tests/test_scheduler.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index d97af00a44a..599143f72f8 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -421,11 +421,7 @@ async def test_queued_remove_add_worker(c, s, a, b): event = Event() fs = c.map(lambda i: event.wait(), range(10)) - await async_wait_for( - lambda: len(s.queued) == 6, - timeout=5, - fail_func=lambda: print(list(s.queued.sorted()), s.tasks), - ) + await async_wait_for(lambda: len(s.queued) == 6, timeout=5) await s.remove_worker(a.address, stimulus_id="fake") assert len(s.queued) == 8 From 3cebe54868c89e70f5408434f1cf8fead843da7b Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Thu, 25 Aug 2022 21:02:59 -0600 Subject: [PATCH 81/98] remove `test_oversaturation_multiple_task_groups` will add it back when we actually implement co-assignment --- distributed/tests/test_scheduler.py | 21 --------------------- 1 file changed, 21 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 599143f72f8..8545dba331c 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -525,27 +525,6 @@ async def _test_saturation_factor(c, s, a, b): _test_saturation_factor() -@pytest.mark.skip("Current queuing does not support co-assignment") -@pytest.mark.parametrize("saturation_factor", [1.0, 2.0, float("inf")]) -@gen_cluster( - client=True, - nthreads=[("", 2), ("", 1)], - config={"distributed.scheduler.worker-saturation": 1.0}, -) -async def test_oversaturation_multiple_task_groups(c, s, a, b, saturation_factor): - s.WORKER_SATURATION = saturation_factor - xs = [delayed(i, name=f"x-{i}") for i in range(9)] - ys = [delayed(i, name=f"y-{i}") for i in range(9)] - zs = [x + y for x, y in zip(xs, ys)] - - await c.gather(c.compute(zs)) - - assert not a.incoming_transfer_log, [l["keys"] for l in a.incoming_transfer_log] - assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] - assert len(a.state.tasks) == 18 - assert len(b.state.tasks) == 9 - - @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) async def test_move_data_over_break_restrictions(client, s, a, b, c): [x] = await client.scatter([1], workers=b.address) From 14dc850c3b5d4cfd626d87d7023d28ebf7fe2ee5 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 26 Aug 2022 19:06:32 -0600 Subject: [PATCH 82/98] Documentation suggestions Co-authored-by: crusaderky --- distributed/distributed-schema.yaml | 5 +++-- distributed/scheduler.py | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 4eed80be539..a110aaa8078 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -126,10 +126,11 @@ properties: Up to worker-saturation * nthreads root tasks are sent to a worker at a time. If `.inf`, all runnable tasks are immediately sent to workers. - Allowing oversaturation (> 1.0) means a worker will start running a new root task as + Allowing oversaturation (> 1.0) means a worker may start running a new root task as soon as it completes the previous, even if there is a higher-priority downstream task to run. This reduces worker idleness, by letting workers do something while waiting for - further instructions from the scheduler, even if it's not the most useful thing. + further instructions from the scheduler, even if it's not the most efficient + thing to do. This generally comes at the expense of increased memory usage. It leads to "wider" (more breadth-first) execution of the graph. diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 43d070aff3c..d7f9fb088eb 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2738,7 +2738,7 @@ def is_rootish(self, ts: TaskState) -> bool: """ Whether ``ts`` is a root or root-like task. - Root-ish tasks are part of a group that's larger than the cluster, + Root-ish tasks are part of a group that's much larger than the cluster, and have few or no dependencies. """ if ts.resource_restrictions or ts.worker_restrictions or ts.host_restrictions: From b36064e0c0d50bd66a170272ff148357d1d1de20 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 26 Aug 2022 18:35:17 -0600 Subject: [PATCH 83/98] `test_graph_execution_width` --- distributed/tests/test_scheduler.py | 51 ++++++++++++++++++++++++++++- 1 file changed, 50 insertions(+), 1 deletion(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index cff865b0396..fe5cc2b9f4e 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -11,7 +11,7 @@ from itertools import product from textwrap import dedent from time import sleep -from typing import Collection +from typing import ClassVar, Collection from unittest import mock import cloudpickle @@ -318,6 +318,55 @@ def big_data(size: int) -> str: assert pids == [n.pid for n in nannies] +@pytest.mark.slow +@pytest.mark.parametrize("gil_sleep", [False, True]) +@gen_cluster( + nthreads=[("", 2)] * 4, + client=True, + config={"distributed.scheduler.worker-saturation": 1.0}, +) +async def test_graph_execution_width(c, s, *workers, gil_sleep): + """ + Test that we don't execute the graph more breadth-first than necessary. + + We shouldn't start loading extra data if we're not going to use it immediately. + The number of parallel work streams match the number of threads. + """ + + class Refcount: + "Track how many instances of this class exist; logs the count at creation and deletion" + + count: ClassVar[int] = 0 + lock: ClassVar[dask.utils.SerializableLock] = dask.utils.SerializableLock() + log: ClassVar[list[int]] = [] + + def __init__(self) -> None: + sleep_time = 0.1 # significantly longer than scheduler<->worker round-trip + if gil_sleep: + start = time() + while time() < start + sleep_time: + pass # burn CPU holding GIL. This will gum up the event loop on the scheduler and workers. + else: + sleep(sleep_time) + + with self.lock: + type(self).count += 1 + self.log.append(self.count) + + def __del__(self): + with self.lock: + self.log.append(self.count) + type(self).count -= 1 + + roots = [delayed(Refcount)() for _ in range(32)] + passthrough = [delayed(slowidentity)(r) for r in roots] + done = [delayed(lambda r: None)(r) for r in passthrough] + + fs = c.compute(done) + await wait(fs) + assert max(Refcount.log) == s.total_nthreads + + @pytest.mark.parametrize("queue", [True, False]) @gen_cluster( client=True, From 5b2bc0245a32384e2928ac5154bc417d8404ffdf Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Fri, 26 Aug 2022 19:04:44 -0600 Subject: [PATCH 84/98] skip `test_root_task_overproduction` on windows I don't understand why it's flaking on windows, but I imagine it's just because memory measurement and process memory overhead behaves differently. It could really just run on linux, but leaving it un-skipped for macOS right now out of convenience for macOS developers to run locally. --- distributed/tests/test_scheduler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index fe5cc2b9f4e..cb743d599f1 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -272,6 +272,7 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] +@pytest.mark.skipif(WINDOWS, reason="Process memory use seems unpredictable on Windows") @pytest.mark.slow @gen_cluster( client=True, From 2952f6bcf54e6b7b7fa95300456c8a9125eeffa8 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Mon, 29 Aug 2022 16:26:14 -0600 Subject: [PATCH 85/98] few more type annotations since they're available --- distributed/scheduler.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index b7e695ed0ff..0c543bd6233 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1729,7 +1729,7 @@ def transition_released_waiting(self, key, stimulus_id): def transition_no_worker_processing(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] - recommendations: dict = {} + recommendations: Recs = {} client_msgs: dict = {} worker_msgs: dict = {} @@ -2539,7 +2539,7 @@ def transition_no_worker_released(self, key, stimulus_id): def transition_waiting_queued(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] - recommendations: dict = {} + recommendations: Recs = {} client_msgs: dict = {} worker_msgs: dict = {} @@ -2562,7 +2562,7 @@ def transition_waiting_queued(self, key, stimulus_id): def transition_waiting_no_worker(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] - recommendations: dict = {} + recommendations: Recs = {} client_msgs: dict = {} worker_msgs: dict = {} @@ -2584,7 +2584,7 @@ def transition_waiting_no_worker(self, key, stimulus_id): def transition_queued_released(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] - recommendations: dict = {} + recommendations: Recs = {} client_msgs: dict = {} worker_msgs: dict = {} @@ -2607,7 +2607,7 @@ def transition_queued_released(self, key, stimulus_id): def transition_queued_processing(self, key, stimulus_id): try: ts: TaskState = self.tasks[key] - recommendations: dict = {} + recommendations: Recs = {} client_msgs: dict = {} worker_msgs: dict = {} From d00ea54ad39dc0d6f8f85f2e61a584d4173c5db9 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Mon, 29 Aug 2022 16:27:54 -0600 Subject: [PATCH 86/98] `decide_worker_rootish_queuing_enabled` no task don't even need to pass it in right now; it's not used --- distributed/scheduler.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 0c543bd6233..65737c81832 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1849,9 +1849,7 @@ def decide_worker_rootish_queuing_disabled( return ws - def decide_worker_rootish_queuing_enabled( - self, ts: TaskState - ) -> WorkerState | None: + def decide_worker_rootish_queuing_enabled(self) -> WorkerState | None: """Pick a worker for a runnable root-ish task, if not all are busy. Picks the least-busy worker out of the ``idle`` workers (idle workers have fewer @@ -1879,7 +1877,8 @@ def decide_worker_rootish_queuing_enabled( # We don't `assert self.is_rootish(ts)` here, because that check is dependent on # cluster size. It's possible a task looked root-ish when it was queued, but the # cluster has since scaled up and it no longer does when coming out of the queue. - # If `is_rootish` changes to a static definition, then add that assertion here. + # If `is_rootish` changes to a static definition, then add that assertion here + # (and actually pass in the task). assert not math.isinf(self.WORKER_SATURATION) if not self.idle: @@ -1986,7 +1985,7 @@ def transition_waiting_processing(self, key, stimulus_id): if not (ws := self.decide_worker_rootish_queuing_disabled(ts)): return {ts.key: "no-worker"}, {}, {} else: - if not (ws := self.decide_worker_rootish_queuing_enabled(ts)): + if not (ws := self.decide_worker_rootish_queuing_enabled()): return {ts.key: "queued"}, {}, {} else: if not (ws := self.decide_worker_non_rootish(ts)): @@ -2615,7 +2614,7 @@ def transition_queued_processing(self, key, stimulus_id): assert not ts.actor, f"Actors can't be queued: {ts}" assert ts in self.queued - if ws := self.decide_worker_rootish_queuing_enabled(ts): + if ws := self.decide_worker_rootish_queuing_enabled(): self.queued.discard(ts) worker_msgs = _add_to_processing(self, ts, ws) # If no worker, task just stays `queued` From 8ba4ced53a4e85af104a1ff48559a436a05705ca Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Mon, 29 Aug 2022 16:44:24 -0600 Subject: [PATCH 87/98] fix `check_idle` for queued tasks --- distributed/scheduler.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 65737c81832..7e7589663c7 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -7488,7 +7488,11 @@ def check_idle(self): self.idle_since = None return - if any([ws.processing for ws in self.workers.values()]) or self.unrunnable: + if ( + self.queued + or self.unrunnable + or any([ws.processing for ws in self.workers.values()]) + ): self.idle_since = None return From 63d863d14e67c511802cb43d027bb3c3805fd0f2 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 17:01:27 -0600 Subject: [PATCH 88/98] _remove_from_processing->_exit_processing_common --- distributed/scheduler.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 7e7589663c7..4b87008daf6 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2128,7 +2128,7 @@ def transition_processing_memory( # NOTE: recommendations for queued tasks are added first, so they'll be popped last, # allowing higher-priority downstream tasks to be transitioned first. # FIXME: this would be incorrect if queued tasks are user-annotated as higher priority. - _remove_from_processing(self, ts, recommendations) + _exit_processing_common(self, ts, recommendations) _add_to_memory( self, ts, ws, recommendations, client_msgs, type=type, typename=typename @@ -2359,7 +2359,7 @@ def transition_processing_released(self, key: str, stimulus_id: str): assert not ts.waiting_on assert ts.state == "processing" - ws = _remove_from_processing(self, ts, recommendations) + ws = _exit_processing_common(self, ts, recommendations) if ws: worker_msgs[ws.address] = [ { @@ -2436,7 +2436,7 @@ def transition_processing_erred( ws = ts.processing_on ws.actors.remove(ts) - _remove_from_processing(self, ts, recommendations) + _exit_processing_common(self, ts, recommendations) ts.erred_on.add(worker) if exception is not None: @@ -7655,7 +7655,7 @@ def _add_to_processing( return {ws.address: [_task_to_msg(state, ts)]} -def _remove_from_processing( +def _exit_processing_common( state: SchedulerState, ts: TaskState, recommendations: Recs ) -> WorkerState | None: """Remove *ts* from the set of processing tasks. From b7704e3b34ec9199a77e0594876a425347b9a367 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 17:18:15 -0600 Subject: [PATCH 89/98] remove `test_queued_tasks_rebalance` --- distributed/tests/test_scheduler.py | 48 ----------------------------- 1 file changed, 48 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index cb743d599f1..34166431f73 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -483,54 +483,6 @@ async def test_queued_remove_add_worker(c, s, a, b): await wait(fs) -@pytest.mark.slow -@gen_cluster( - client=True, - nthreads=[("", 2)] * 2, - config={"distributed.scheduler.worker-saturation": 1.0}, -) -async def test_queued_tasks_rebalance(client, s, a, b): - """ - Test that all queued tasks complete as workers come and go. - - Does not test how well balanced the load was. - """ - roots1 = [delayed(slowinc)(i) for i in range(40)] - roots2 = [delayed(slowinc)(i, delay=0.01) for i in range(len(roots1))] - combined = [x + y for x, y in zip(roots1, roots2)] - agg = [sum(xs) for xs in partition(4, combined)] - - fs = client.compute(agg) - - while len(a.data) + len(b.data) < len(agg) * 0.25: - await asyncio.sleep(0.01) - - # Add a new worker - async with Worker(s.address, nthreads=2) as c: - while not c.data: - await asyncio.sleep(0.01) - - # Now add another - async with Worker(s.address, nthreads=1) as d: - while not d.data: - await asyncio.sleep(0.01) - - # Remove an existing worker - await a.close() - - while len(b.data) + len(c.data) + len(d.data) < len(agg) * 0.75: - await asyncio.sleep(0.01) - - # And a new one - await d.close() - - await client.gather(fs) - assert a.state.tasks - assert b.state.tasks - assert c.state.tasks - assert d.state.tasks - - @pytest.mark.parametrize( "saturation, expected_task_counts", [ From 00b54e732014a2cf42e3f5a182d1678f09bf04f9 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 17:20:10 -0600 Subject: [PATCH 90/98] `scheduler.allowed-failures` instead of PIDs --- distributed/tests/test_scheduler.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 34166431f73..671dfe41819 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -284,6 +284,7 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): "distributed.worker.memory.target": False, "distributed.worker.memory.spill": False, "distributed.scheduler.work-stealing": False, + "distributed.scheduler.allowed-failures": 0, # don't allow nannies to restart }, ) async def test_root_task_overproduction(c, s, *nannies): @@ -291,7 +292,6 @@ async def test_root_task_overproduction(c, s, *nannies): Workload that would run out of memory and kill workers if >2 root tasks were ever in memory at once on a worker. """ - pids = [n.pid for n in nannies] @delayed(pure=True) # type: ignore def big_data(size: int) -> str: @@ -315,9 +315,6 @@ def big_data(size: int) -> str: await c.compute(final) - # No restarts - assert pids == [n.pid for n in nannies] - @pytest.mark.slow @pytest.mark.parametrize("gil_sleep", [False, True]) From 12207e6bbcd8a176195c463c6a996c03bdd90765 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 17:45:23 -0600 Subject: [PATCH 91/98] test_root_task_overproduction->test_near_memory_limit_workload --- distributed/tests/test_scheduler.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 671dfe41819..3826c1849a4 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -287,10 +287,15 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): "distributed.scheduler.allowed-failures": 0, # don't allow nannies to restart }, ) -async def test_root_task_overproduction(c, s, *nannies): +async def test_near_memory_limit_workload(c, s, *nannies): """ - Workload that would run out of memory and kill workers if >2 root tasks were - ever in memory at once on a worker. + Integration test: a workload close to worker memory limit, which might fail with bad scheduling decisions. + + Each worker has 2 threads. The data is sized such that if >2 pieces of data are + ever in memory at once, the worker would exceed its memory limit and be killed. + + The scheduler must decide when and where to schedule tasks so extra pieces of data + are not produced, and data is not transferred. """ @delayed(pure=True) # type: ignore From 02c98b3effcc52de436308b6a300a778acffbc71 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 18:47:16 -0600 Subject: [PATCH 92/98] remove all sleeps from `graph_execution_width` --- distributed/tests/test_scheduler.py | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 3826c1849a4..508b09a7107 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -322,13 +322,12 @@ def big_data(size: int) -> str: @pytest.mark.slow -@pytest.mark.parametrize("gil_sleep", [False, True]) @gen_cluster( nthreads=[("", 2)] * 4, client=True, config={"distributed.scheduler.worker-saturation": 1.0}, ) -async def test_graph_execution_width(c, s, *workers, gil_sleep): +async def test_graph_execution_width(c, s, *workers): """ Test that we don't execute the graph more breadth-first than necessary. @@ -344,14 +343,6 @@ class Refcount: log: ClassVar[list[int]] = [] def __init__(self) -> None: - sleep_time = 0.1 # significantly longer than scheduler<->worker round-trip - if gil_sleep: - start = time() - while time() < start + sleep_time: - pass # burn CPU holding GIL. This will gum up the event loop on the scheduler and workers. - else: - sleep(sleep_time) - with self.lock: type(self).count += 1 self.log.append(self.count) @@ -362,8 +353,9 @@ def __del__(self): type(self).count -= 1 roots = [delayed(Refcount)() for _ in range(32)] - passthrough = [delayed(slowidentity)(r) for r in roots] - done = [delayed(lambda r: None)(r) for r in passthrough] + passthrough1 = [delayed(slowidentity)(r, delay=0) for r in roots] + passthrough2 = [delayed(slowidentity)(r, delay=0) for r in passthrough1] + done = [delayed(lambda r: None)(r) for r in passthrough2] fs = c.compute(done) await wait(fs) From 2b3f6ae9ffbc9bab2a0459e9b727b6f6cb847a0a Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 18:52:55 -0600 Subject: [PATCH 93/98] remove `test_near_memory_limit_workload` feeling pretty good about just `test_graph_execution_width` --- distributed/tests/test_scheduler.py | 60 +---------------------------- 1 file changed, 2 insertions(+), 58 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 508b09a7107..6830cdcd6f4 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -17,19 +17,12 @@ import cloudpickle import psutil import pytest -from tlz import concat, first, merge, partition, valmap +from tlz import concat, first, merge, valmap from tornado.ioloop import IOLoop, PeriodicCallback import dask from dask import delayed -from dask.utils import ( - apply, - format_bytes, - parse_timedelta, - stringify, - tmpfile, - typename, -) +from dask.utils import apply, parse_timedelta, stringify, tmpfile, typename from distributed import ( CancelledError, @@ -272,55 +265,6 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] -@pytest.mark.skipif(WINDOWS, reason="Process memory use seems unpredictable on Windows") -@pytest.mark.slow -@gen_cluster( - client=True, - nthreads=[("", 2)] * 2, - worker_kwargs={"memory_limit": "1.0GiB"}, - Worker=Nanny, - config={ - "distributed.scheduler.worker-saturation": 1.0, - "distributed.worker.memory.target": False, - "distributed.worker.memory.spill": False, - "distributed.scheduler.work-stealing": False, - "distributed.scheduler.allowed-failures": 0, # don't allow nannies to restart - }, -) -async def test_near_memory_limit_workload(c, s, *nannies): - """ - Integration test: a workload close to worker memory limit, which might fail with bad scheduling decisions. - - Each worker has 2 threads. The data is sized such that if >2 pieces of data are - ever in memory at once, the worker would exceed its memory limit and be killed. - - The scheduler must decide when and where to schedule tasks so extra pieces of data - are not produced, and data is not transferred. - """ - - @delayed(pure=True) # type: ignore - def big_data(size: int) -> str: - return "x" * size - - available = await c.run( - lambda dask_worker: dask_worker.memory_manager.memory_limit - - dask_worker.monitor.get_process_memory() - ) - print({k: format_bytes(v) for k, v in available.items()}) - min_available = min(available.values()) - # leave room so 2 can comfortably fit, but 3 can't - task_size = round(min_available / 2.9) - print(f"task size: {format_bytes(task_size)}") - - roots = [big_data(task_size, dask_key_name=f"root-{i}") for i in range(16)] - passthrough = [delayed(slowidentity)(x) for x in roots] - memory_consumed = [delayed(len)(x) for x in passthrough] - reduction = [sum(sizes) for sizes in partition(4, memory_consumed)] - final = sum(reduction) - - await c.compute(final) - - @pytest.mark.slow @gen_cluster( nthreads=[("", 2)] * 4, From 5e4d53d38344f367965865d1d879676069d9853a Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 23:16:44 -0600 Subject: [PATCH 94/98] `handle_worker_status_change` in `retire_workers` Using it as an API saves having to manage `running` and `idle` in multiple places --- distributed/scheduler.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 4b87008daf6..6dd88e08336 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5104,13 +5104,13 @@ def handle_long_running( self.check_idle_saturated(ws) def handle_worker_status_change( - self, status: str, worker: str, stimulus_id: str + self, status: str | Status, worker: str | WorkerState, stimulus_id: str ) -> None: - ws = self.workers.get(worker) + ws = self.workers.get(worker) if isinstance(worker, str) else worker if not ws: return prev_status = ws.status - ws.status = Status.lookup[status] # type: ignore + ws.status = Status[status] if isinstance(status, str) else status if ws.status == prev_status: return @@ -6450,9 +6450,9 @@ async def retire_workers( # 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) - self.idle.pop(ws.address, None) + self.handle_worker_status_change( + Status.closing_gracefully, ws, stimulus_id + ) # 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( From acc524f20b66c01c2aef398e79b96c1ebd8f3069 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Tue, 30 Aug 2022 23:39:44 -0600 Subject: [PATCH 95/98] avoid flaky `test_graph_execution_width` hesitant on this, but I don't want to introduce a flaky test --- distributed/tests/test_scheduler.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index c1da7ffe0f6..5f122f2b8d0 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -303,7 +303,9 @@ def __del__(self): fs = c.compute(done) await wait(fs) - assert max(Refcount.log) == s.total_nthreads + # NOTE: the max should normally equal `total_nthreads`. But some macOS CI machines + # are slow enough that they aren't able to reach the full parallelism of 8 threads. + assert max(Refcount.log) <= s.total_nthreads @pytest.mark.parametrize("queue", [True, False]) From ba336b9cb0e2cd247fd776583b6f3e6408907cf3 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 31 Aug 2022 00:34:19 -0600 Subject: [PATCH 96/98] fix test_decide_worker_coschedule_order_binary_op --- distributed/tests/test_scheduler.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 5f122f2b8d0..dd0a4343bea 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -261,8 +261,12 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): await c.gather(c.compute(zs)) - assert not a.incoming_transfer_log, [l["keys"] for l in a.incoming_transfer_log] - assert not b.incoming_transfer_log, [l["keys"] for l in b.incoming_transfer_log] + assert not a.state.incoming_transfer_log, [ + l["keys"] for l in a.state.incoming_transfer_log + ] + assert not b.state.incoming_transfer_log, [ + l["keys"] for l in b.state.incoming_transfer_log + ] @pytest.mark.slow From 9d99d7422bb9bb76ac1f8ca7ab5f8070e9371bb6 Mon Sep 17 00:00:00 2001 From: Gabe Joseph Date: Wed, 31 Aug 2022 00:37:28 -0600 Subject: [PATCH 97/98] fixup! `handle_worker_status_change` --- distributed/scheduler.py | 2 +- distributed/tests/test_worker.py | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 6dd88e08336..8ffb215025f 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -5119,7 +5119,7 @@ def handle_worker_status_change( { "action": "worker-status-change", "prev-status": prev_status.name, - "status": status, + "status": ws.status.name, }, ) logger.debug(f"Worker status {prev_status.name} -> {status} - {ws}") diff --git a/distributed/tests/test_worker.py b/distributed/tests/test_worker.py index 0103808cf55..1fe036a371c 100644 --- a/distributed/tests/test_worker.py +++ b/distributed/tests/test_worker.py @@ -3116,6 +3116,11 @@ async def test_worker_status_sync(s, a): "prev-status": "paused", "status": "running", }, + { + "action": "worker-status-change", + "prev-status": "running", + "status": "closing_gracefully", + }, {"action": "remove-worker", "processing-tasks": {}}, {"action": "retired"}, ] From 093d7dcbf11ff6b26032e35f8aabef4ae180b778 Mon Sep 17 00:00:00 2001 From: fjetter Date: Wed, 31 Aug 2022 11:06:43 +0200 Subject: [PATCH 98/98] Fix merge conflict of renaming transfer log --- distributed/tests/test_scheduler.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index dd0a4343bea..f83e5f45bb7 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -261,12 +261,8 @@ async def test_decide_worker_coschedule_order_binary_op(c, s, a, b, ngroups): await c.gather(c.compute(zs)) - assert not a.state.incoming_transfer_log, [ - l["keys"] for l in a.state.incoming_transfer_log - ] - assert not b.state.incoming_transfer_log, [ - l["keys"] for l in b.state.incoming_transfer_log - ] + assert not a.transfer_incoming_log, [l["keys"] for l in a.transfer_incoming_log] + assert not b.transfer_incoming_log, [l["keys"] for l in b.transfer_incoming_log] @pytest.mark.slow