Skip to content

Commit

Permalink
Improve work stealing for scaling situations
Browse files Browse the repository at this point in the history
  • Loading branch information
fjetter committed Jun 15, 2021
1 parent ec9b569 commit dc9586d
Show file tree
Hide file tree
Showing 2 changed files with 20 additions and 15 deletions.
15 changes: 10 additions & 5 deletions distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -2340,8 +2340,9 @@ def decide_worker(self, ts: TaskState) -> WorkerState:
if ts._dependencies or valid_workers is not None:
ws = decide_worker(
ts,
self._workers_dv.values(),
set(self._workers_dv.values()),
valid_workers,
set(self._idle_dv),
partial(self.worker_objective, ts),
)
else:
Expand Down Expand Up @@ -7471,7 +7472,11 @@ def _reevaluate_occupancy_worker(state: SchedulerState, ws: WorkerState):
@cfunc
@exceptval(check=False)
def decide_worker(
ts: TaskState, all_workers, valid_workers: set, objective
ts: TaskState,
all_workers: set,
valid_workers: Optional[set],
idle: set,
objective,
) -> WorkerState:
"""
Decide which worker should take task *ts*.
Expand All @@ -7495,19 +7500,19 @@ def decide_worker(
candidates: set
assert all([dts._who_has for dts in deps])
if ts._actor:
candidates = set(all_workers)
candidates = all_workers
else:
candidates = {wws for dts in deps for wws in dts._who_has}
if valid_workers is None:
if not candidates:
candidates = set(all_workers)
candidates = all_workers
else:
candidates &= valid_workers
if not candidates:
candidates = valid_workers
if not candidates:
if ts._loose_restrictions:
ws = decide_worker(ts, all_workers, None, objective)
ws = decide_worker(ts, all_workers, None, idle, objective)
return ws

ncandidates: Py_ssize_t = len(candidates)
Expand Down
20 changes: 10 additions & 10 deletions distributed/stealing.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
import logging
from collections import defaultdict, deque
from functools import partial
from math import log2
from time import time

Expand Down Expand Up @@ -113,27 +114,21 @@ def steal_time_ratio(self, ts):
For example a result of zero implies a task without dependencies.
level: The location within a stealable list to place this value
"""
if not ts.dependencies: # no dependencies fast path
return 0, 0

split = ts.prefix.name
if split in fast_tasks:
return None, None

ws = ts.processing_on
compute_time = ws.processing[ts]
if compute_time < 0.005: # 5ms, just give up
return None, None

nbytes = ts.get_nbytes_deps()
transfer_time = nbytes / self.scheduler.bandwidth + LATENCY
cost_multiplier = transfer_time / compute_time
if cost_multiplier > 100:
return None, None

level = int(round(log2(cost_multiplier) + 6))
if level < 1:
level = 1

level = min(len(self.cost_multipliers) - 1, level)

return cost_multiplier, level

Expand Down Expand Up @@ -344,7 +339,10 @@ def maybe_move_task(level, ts, sat, idl, duration, cost_multiplier):
thieves = idle
if not thieves:
break
thief = thieves[i % len(thieves)]

thief = min(
thieves, key=partial(self.scheduler.worker_objective, ts)
)

duration = sat.processing.get(ts)
if duration is None:
Expand Down Expand Up @@ -380,7 +378,9 @@ def maybe_move_task(level, ts, sat, idl, duration, cost_multiplier):
thieves = idle
if not thieves:
continue
thief = thieves[i % len(thieves)]
thief = min(
thieves, key=partial(self.scheduler.worker_objective, ts)
)
duration = sat.processing[ts]

maybe_move_task(
Expand Down

0 comments on commit dc9586d

Please sign in to comment.