Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Allow unknown tasks to be stolen #5572

Merged
merged 4 commits into from
Dec 10, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -2813,7 +2813,7 @@ def transition_processing_memory(
if tts._processing_on:
self.set_duration_estimate(tts, tts._processing_on)
if steal:
steal.put_key_in_stealable(tts)
steal.recalculate_cost(tts)

############################
# Update State Information #
Expand Down
43 changes: 37 additions & 6 deletions distributed/stealing.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
from __future__ import annotations

import asyncio
import logging
from collections import defaultdict, deque
from math import log2
Expand Down Expand Up @@ -63,14 +64,11 @@ def __init__(self, scheduler):
for worker in scheduler.workers:
self.add_worker(worker=worker)

callback_time = parse_timedelta(
self._callback_time = parse_timedelta(
dask.config.get("distributed.scheduler.work-stealing-interval"),
default="ms",
)
# `callback_time` is in milliseconds
pc = PeriodicCallback(callback=self.balance, callback_time=callback_time * 1000)
self._pc = pc
self.scheduler.periodic_callbacks["stealing"] = pc
self.scheduler.add_plugin(self)
self.scheduler.extensions["stealing"] = self
self.scheduler.events["stealing"] = deque(maxlen=100000)
Expand All @@ -79,9 +77,36 @@ def __init__(self, scheduler):
self.in_flight = dict()
# { worker state: occupancy }
self.in_flight_occupancy = defaultdict(lambda: 0)
self._in_flight_event = asyncio.Event()

self.scheduler.stream_handlers["steal-response"] = self.move_task_confirm

async def start(self, scheduler=None):
"""Start the background coroutine to balance the tasks on the cluster.
Idempotent.
The scheduler argument is ignored. It is merely required to satisify the
plugin interface. Since this class is simultaneouly an extension, the
scheudler instance is already registered during initialization
"""
if "stealing" in self.scheduler.periodic_callbacks:
return
pc = PeriodicCallback(
callback=self.balance, callback_time=self._callback_time * 1000
)
pc.start()
self.scheduler.periodic_callbacks["stealing"] = pc
self._in_flight_event.set()

async def stop(self):
"""Stop the background task balancing tasks on the cluster.
This will block until all currently running stealing requests are
finished. Idempotent
"""
pc = self.scheduler.periodic_callbacks.pop("stealing", None)
if pc:
pc.stop()
await self._in_flight_event.wait()

def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
"""
A very verbose dictionary representation for debugging purposes.
Expand Down Expand Up @@ -118,7 +143,10 @@ def remove_worker(self, scheduler=None, worker=None):
del self.stealable[worker]

def teardown(self):
self._pc.stop()
pcs = self.scheduler.periodic_callbacks
if "stealing" in pcs:
pcs["stealing"].stop()
del pcs["stealing"]

def transition(
self, key, start, finish, compute_start=None, compute_stop=None, *args, **kwargs
Expand All @@ -137,6 +165,7 @@ def transition(
self.in_flight_occupancy[victim] += d["victim_duration"]
if not self.in_flight:
self.in_flight_occupancy.clear()
self._in_flight_event.set()

def recalculate_cost(self, ts):
if ts not in self.in_flight:
Expand Down Expand Up @@ -177,7 +206,7 @@ def steal_time_ratio(self, ts):
level: The location within a stealable list to place this value
"""
split = ts.prefix.name
if split in fast_tasks or split in self.scheduler.unknown_durations:
if split in fast_tasks:
return None, None

if not ts.dependencies: # no dependencies fast path
Expand Down Expand Up @@ -233,6 +262,7 @@ def move_task_request(self, ts, victim, thief) -> str:
"thief_duration": thief_duration,
"stimulus_id": stimulus_id,
}
self._in_flight_event.clear()

self.in_flight_occupancy[victim] -= victim_duration
self.in_flight_occupancy[thief] += thief_duration
Expand Down Expand Up @@ -274,6 +304,7 @@ async def move_task_confirm(self, *, key, state, stimulus_id, worker=None):

if not self.in_flight:
self.in_flight_occupancy.clear()
self._in_flight_event.set()

if self.scheduler.validate:
assert ts.processing_on == victim
Expand Down
4 changes: 2 additions & 2 deletions distributed/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -4306,7 +4306,7 @@ async def test_retire_many_workers(c, s, *workers):
config={"distributed.scheduler.default-task-durations": {"f": "10ms"}},
)
async def test_weight_occupancy_against_data_movement(c, s, a, b):
s.extensions["stealing"]._pc.callback_time = 1000000
await s.extensions["stealing"].stop()

def f(x, y=0, z=0):
sleep(0.01)
Expand All @@ -4329,7 +4329,7 @@ def f(x, y=0, z=0):
config={"distributed.scheduler.default-task-durations": {"f": "10ms"}},
)
async def test_distribute_tasks_by_nthreads(c, s, a, b):
s.extensions["stealing"]._pc.callback_time = 1000000
await s.extensions["stealing"].stop()

def f(x, y=0):
sleep(0.01)
Expand Down
2 changes: 1 addition & 1 deletion distributed/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -1020,7 +1020,7 @@ async def test_balance_many_workers(c, s, *workers):
@nodebug
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 30)
async def test_balance_many_workers_2(c, s, *workers):
s.extensions["stealing"]._pc.callback_time = 100000000
await s.extensions["stealing"].stop()
futures = c.map(slowinc, range(90), delay=0.2)
await wait(futures)
assert {len(w.has_what) for w in s.workers.values()} == {3}
Expand Down
Loading