Skip to content

Commit

Permalink
retire_workers to use AMM
Browse files Browse the repository at this point in the history
  • Loading branch information
crusaderky committed Dec 10, 2021
1 parent 96a4cea commit b57da8d
Show file tree
Hide file tree
Showing 7 changed files with 582 additions and 120 deletions.
147 changes: 146 additions & 1 deletion distributed/active_memory_manager.py
Expand Up @@ -122,7 +122,7 @@ def add_policy(self, policy: ActiveMemoryManagerPolicy) -> None:

def run_once(self) -> None:
"""Run all policies once and asynchronously (fire and forget) enact their
recommendations to replicate/drop keys
recommendations to replicate/drop tasks
"""
with log_errors():
# This should never fail since this is a synchronous method
Expand Down Expand Up @@ -198,6 +198,9 @@ def _find_recipient(
drops), or None if no eligible candidates are available.
"""
if ts.state != "memory":
logger.debug(
"(replicate, %s, %s) rejected: ts.state = %s", ts, candidates, ts.state
)
return None
if candidates is None:
candidates = self.scheduler.running.copy()
Expand All @@ -207,6 +210,9 @@ def _find_recipient(
candidates -= ts.who_has
candidates -= pending_repl
if not candidates:
logger.debug(
"(replicate, %s, %s) rejected: no valid candidates", ts, candidates
)
return None

# Select candidate with the lowest memory usage
Expand All @@ -230,6 +236,9 @@ def _find_dropper(
drops), or None if no eligible candidates are available.
"""
if len(ts.who_has) - len(pending_drop) < 2:
logger.debug(
"(drop, %s, %s) rejected: less than 2 replicas exist", ts, candidates
)
return None
if candidates is None:
candidates = ts.who_has.copy()
Expand All @@ -238,6 +247,7 @@ def _find_dropper(
candidates -= pending_drop
candidates -= {waiter_ts.processing_on for waiter_ts in ts.waiters}
if not candidates:
logger.debug("(drop, %s, %s) rejected: no valid candidates", ts, candidates)
return None

# Select candidate with the highest memory usage.
Expand Down Expand Up @@ -383,6 +393,7 @@ class ReduceReplicas(ActiveMemoryManagerPolicy):
"""

def run(self):
""""""
nkeys = 0
ndrop = 0

Expand Down Expand Up @@ -412,3 +423,137 @@ def run(self):
ndrop,
nkeys,
)


class RetireWorker(ActiveMemoryManagerPolicy):
"""Replicate somewhere else all unique in-memory tasks on a worker, preparing for
its shutdown.
At any given time, the AMM may have registered multiple instances of this policy,
one for each worker currently being retired - meaning that most of the time no
instances will be registered at all. For this reason, this policy doesn't figure in
the dask config (:file:`distributed.yaml`). Instances are added by
:meth:`distributed.Scheduler.retire_workers` and automatically remove themselves
once the worker has been retired. If the AMM is disabled in the dask config,
:meth:`~distributed.Scheduler.retire_workers` will start a temporary ad-hoc one.
**Failure condition**
There may not be any suitable workers to receive the tasks from the retiring worker.
This happens in two use cases:
1. This is the only worker in the cluster, or
2. All workers are either paused or being retired at the same time
In either case, this policy will fail to move out all keys and
Scheduler.retire_workers will abort the retirement. The flag ``no_recipients`` will
be raised.
There is a third use case, where a task fails to be replicated away for whatever
reason; in this case we'll just wait for the next AMM iteration and try again
(possibly with a different receiving worker, e.g. if the receiving worker was
hung but not yet declared dead).
**Retiring a worker with spilled tasks**
On its very first iteration, this policy suggests that other workers should fetch
all unique in-memory tasks. Frequently, this means that in the next few moments the
worker to be retired will be bombarded by :meth:`distributed.worker.Worker.get_data`
calls from the rest of the cluster. This can be a problem if most of the managed
memory of the worker has been spilled out, as it could send the worker above the
terminate threshold. Two measures are in place in order to prevent this:
- At every iteration, this policy drops all tasks that have already been replicated
somewhere else. This makes room for further tasks to be moved out of the spill
file in order to be replicated onto another worker.
- Once a worker passes the ``pause`` threshold,
:meth:`~distributed.worker.Worker.get_data` throttles the number of outgoing
connections to 1.
Parameters
==========
address: str
URI of the worker to be retired
"""

address: str
no_recipients: bool

def __init__(self, address: str):
self.address = address
self.no_recipients = False

def __repr__(self) -> str:
return f"RetireWorker({self.address}, done={self.done})"

def run(self):
""""""
ws = self.manager.scheduler.workers.get(self.address)
if ws is None:
self.manager.policies.remove(self)
return

nrepl = 0
nno_rec = 0

for ts in ws.has_what:
pending_repl, pending_drop = self.manager.pending[ts]

if len(ts.who_has) > 1:
# There are already replicas of this key on other workers.
# Use cases:
# 1. The suggestion is accepted by the AMM and by the Worker.
# The replica on this worker is dropped.
# 2. The suggestion is accepted by the AMM, but rejected by the Worker.
# We'll try again at the next AMM iteration.
# 3. The suggestion is rejected by the AMM, because another policy
# (e.g. ReduceReplicas) already suggested the same for this worker
# 4. The suggestion is rejected by the AMM, because all replicas of the
# key are on workers being retired and the other RetireWorker
# instances already asked the same suggestion. We need to deal with
# this case and create a replica elsewhere.
if ws in pending_drop:
continue # Use case 3
drop_ws = (yield "drop", ts, {ws})
if drop_ws:
continue # Use case 1 or 2

# Either the worker holds the only replica or all replicas are being held
# by workers that are being retired
nrepl += 1
if not pending_repl:
rec_ws = (yield "replicate", ts, None)
if not rec_ws:
# replication was rejected by the AMM (see _find_recipient)
nno_rec += 1

if nno_rec:
# All workers are paused or closing_gracefully.
# Scheduler.retire_workers will read this flag and exit immediately.
# TODO after we implement the automatic transition of workers from paused
# to closing_gracefully after a timeout expires, we should revisit this
# code to wait for paused workers and only exit immediately if all
# workers are in closing_gracefully status.
self.no_recipients = True
logger.warning(
f"Tried retiring worker {self.address}, but {nno_rec} tasks could not "
"be moved as there are no suitable workers to receive them. "
"The worker will not be retired."
)
self.manager.policies.remove(self)
elif nrepl:
logger.info(
f"Retiring worker {self.address}; {nrepl} keys are being moved away.",
)
else:
logger.info(
f"Retiring worker {self.address}; no unique keys need to be moved away."
)
self.manager.policies.remove(self)

def done(self) -> bool:
"""Return True if it is safe to close the worker down; False otherwise"""
ws = self.manager.scheduler.workers.get(self.address)
if ws is None:
return True
return all(len(ts.who_has) > 1 for ts in ws.has_what)

0 comments on commit b57da8d

Please sign in to comment.