Skip to content

Commit 5589049

Browse files
Increase default work-stealing interval by 10x (#8997)
1 parent fd3722d commit 5589049

File tree

4 files changed

+90
-22
lines changed

4 files changed

+90
-22
lines changed

distributed/dashboard/tests/test_scheduler_bokeh.py

+3-1
Original file line numberDiff line numberDiff line change
@@ -137,7 +137,9 @@ async def test_counters(c, s, a, b):
137137
await asyncio.sleep(0.01)
138138

139139

140-
@gen_cluster(client=True)
140+
@gen_cluster(
141+
client=True, config={"distributed.scheduler.work-stealing-interval": "100ms"}
142+
)
141143
async def test_stealing_events(c, s, a, b):
142144
se = StealingEvents(s)
143145

distributed/distributed.yaml

+1-1
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ distributed:
2121
idle-timeout: null # Shut down after this duration, like "1h" or "30 minutes"
2222
no-workers-timeout: null # If a task remains unrunnable for longer than this, it fails.
2323
work-stealing: True # workers should steal tasks from each other
24-
work-stealing-interval: 100ms # Callback time for work stealing
24+
work-stealing-interval: 1s # Callback time for work stealing
2525
worker-saturation: 1.1 # Send this fraction of nthreads root tasks to workers
2626
rootish-taskgroup: 5 # number of dependencies of a rootish tg
2727
rootish-taskgroup-dependencies: 5 # number of dependencies of the dependencies of the rootish tg

distributed/http/scheduler/tests/test_stealing_http.py

+6-2
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,9 @@ async def test_prometheus(c, s, a, b):
2525
assert active_metrics == expected_metrics
2626

2727

28-
@gen_cluster(client=True)
28+
@gen_cluster(
29+
client=True, config={"distributed.scheduler.work-stealing-interval": "100ms"}
30+
)
2931
async def test_prometheus_collect_count_total_by_cost_multipliers(c, s, a, b):
3032
pytest.importorskip("prometheus_client")
3133

@@ -58,7 +60,9 @@ async def fetch_metrics_by_cost_multipliers():
5860
assert count == expected_count
5961

6062

61-
@gen_cluster(client=True)
63+
@gen_cluster(
64+
client=True, config={"distributed.scheduler.work-stealing-interval": "100ms"}
65+
)
6266
async def test_prometheus_collect_cost_total_by_cost_multipliers(c, s, a, b):
6367
pytest.importorskip("prometheus_client")
6468

distributed/tests/test_steal.py

+80-18
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,11 @@ async def test_work_stealing(c, s, a, b):
7272
assert len(b.data) > 10
7373

7474

75-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
75+
@gen_cluster(
76+
client=True,
77+
nthreads=[("127.0.0.1", 1)] * 2,
78+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
79+
)
7680
async def test_dont_steal_expensive_data_fast_computation(c, s, a, b):
7781
np = pytest.importorskip("numpy")
7882

@@ -91,7 +95,11 @@ async def test_dont_steal_expensive_data_fast_computation(c, s, a, b):
9195
assert len(a.data) == 12
9296

9397

94-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
98+
@gen_cluster(
99+
client=True,
100+
nthreads=[("127.0.0.1", 1)] * 2,
101+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
102+
)
95103
async def test_steal_cheap_data_slow_computation(c, s, a, b):
96104
x = c.submit(slowinc, 100, delay=0.1) # learn that slowinc is slow
97105
await wait(x)
@@ -104,7 +112,11 @@ async def test_steal_cheap_data_slow_computation(c, s, a, b):
104112

105113

106114
@pytest.mark.slow
107-
@gen_cluster(client=True, nthreads=[("", 1)] * 2, config=NO_AMM)
115+
@gen_cluster(
116+
client=True,
117+
nthreads=[("", 1)] * 2,
118+
config={"distributed.scheduler.work-stealing-interval": "100ms", **NO_AMM},
119+
)
108120
async def test_steal_expensive_data_slow_computation(c, s, a, b):
109121
np = pytest.importorskip("numpy")
110122

@@ -121,7 +133,11 @@ async def test_steal_expensive_data_slow_computation(c, s, a, b):
121133
assert b.data # not empty
122134

123135

124-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 10, config=NO_AMM)
136+
@gen_cluster(
137+
client=True,
138+
nthreads=[("127.0.0.1", 1)] * 10,
139+
config={"distributed.scheduler.work-stealing-interval": "100ms", **NO_AMM},
140+
)
125141
async def test_worksteal_many_thieves(c, s, *workers):
126142
x = c.submit(slowinc, -1, delay=0.1)
127143
await x
@@ -283,7 +299,11 @@ async def test_eventually_steal_unknown_functions(c, s, a, b):
283299

284300

285301
@pytest.mark.skip(reason="")
286-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
302+
@gen_cluster(
303+
client=True,
304+
nthreads=[("127.0.0.1", 1)] * 3,
305+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
306+
)
287307
async def test_steal_related_tasks(e, s, a, b, c):
288308
futures = e.map(
289309
slowinc, range(20), delay=0.05, workers=a.address, allow_other_workers=True
@@ -299,7 +319,11 @@ async def test_steal_related_tasks(e, s, a, b, c):
299319
assert nearby > 10
300320

301321

302-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 10)
322+
@gen_cluster(
323+
client=True,
324+
nthreads=[("127.0.0.1", 1)] * 10,
325+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
326+
)
303327
async def test_dont_steal_fast_tasks_compute_time(c, s, *workers):
304328
def do_nothing(x, y=None):
305329
pass
@@ -315,7 +339,11 @@ def do_nothing(x, y=None):
315339
assert len(s.workers[workers[0].address].has_what) == len(xs) + len(futures)
316340

317341

318-
@gen_cluster(client=True, nthreads=[("", 1)])
342+
@gen_cluster(
343+
client=True,
344+
nthreads=[("", 1)],
345+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
346+
)
319347
async def test_dont_steal_fast_tasks_blocklist(c, s, a):
320348
async with BlockedGetData(s.address) as b:
321349
# create a dependency
@@ -358,7 +386,11 @@ def fast_blocked(i, x):
358386
assert ts.who_has == {ws_a}
359387

360388

361-
@gen_cluster(client=True, nthreads=[("", 1)], config=NO_AMM)
389+
@gen_cluster(
390+
client=True,
391+
nthreads=[("", 1)],
392+
config={"distributed.scheduler.work-stealing-interval": "100ms", **NO_AMM},
393+
)
362394
async def test_new_worker_steals(c, s, a):
363395
await wait(c.submit(slowinc, 1, delay=0.01))
364396

@@ -380,7 +412,10 @@ async def test_new_worker_steals(c, s, a):
380412
assert b.data
381413

382414

383-
@gen_cluster(client=True)
415+
@gen_cluster(
416+
client=True,
417+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
418+
)
384419
async def test_work_steal_allow_other_workers(c, s, a, b):
385420
# Note: this test also verifies the baseline for all other tests below
386421
futures = c.map(
@@ -397,7 +432,10 @@ async def test_work_steal_allow_other_workers(c, s, a, b):
397432
assert result == sum(map(inc, range(100)))
398433

399434

400-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2)])
435+
@gen_cluster(
436+
client=True,
437+
nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2)],
438+
)
401439
async def test_dont_steal_worker_restrictions(c, s, a, b):
402440
futures = c.map(slowinc, range(100), delay=0.05, workers=a.address)
403441

@@ -504,7 +542,11 @@ async def test_steal_resource_restrictions(c, s, a):
504542
assert 20 < len(a.state.tasks) < 80
505543

506544

507-
@gen_cluster(client=True, nthreads=[("", 1, {"resources": {"A": 2, "C": 1}})])
545+
@gen_cluster(
546+
client=True,
547+
nthreads=[("", 1, {"resources": {"A": 2, "C": 1}})],
548+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
549+
)
508550
async def test_steal_resource_restrictions_asym_diff(c, s, a):
509551
# See https://github.com/dask/distributed/issues/5565
510552
future = c.submit(slowinc, 1, delay=0.1, workers=a.address)
@@ -541,7 +583,11 @@ def slow(x):
541583
assert max(durations) / min(durations) < 3
542584

543585

544-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 4)] * 2)
586+
@gen_cluster(
587+
client=True,
588+
nthreads=[("127.0.0.1", 4)] * 2,
589+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
590+
)
545591
async def test_dont_steal_executing_tasks(c, s, a, b):
546592
futures = c.map(
547593
slowinc, range(4), delay=0.1, workers=a.address, allow_other_workers=True
@@ -797,7 +843,10 @@ async def test_restart(c, s, a, b):
797843
assert not any(x for L in steal.stealable.values() for x in L)
798844

799845

800-
@gen_cluster(client=True)
846+
@gen_cluster(
847+
client=True,
848+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
849+
)
801850
async def test_steal_twice(c, s, a, b):
802851
x = c.submit(inc, 1, workers=a.address)
803852
await wait(x)
@@ -841,7 +890,10 @@ async def test_steal_twice(c, s, a, b):
841890
@gen_cluster(
842891
client=True,
843892
nthreads=[("", 1)] * 3,
844-
config={"distributed.worker.memory.pause": False},
893+
config={
894+
"distributed.worker.memory.pause": False,
895+
"distributed.scheduler.work-stealing-interval": "100ms",
896+
},
845897
)
846898
async def test_paused_workers_must_not_steal(c, s, w1, w2, w3):
847899
w2.status = Status.paused
@@ -859,7 +911,10 @@ async def test_paused_workers_must_not_steal(c, s, w1, w2, w3):
859911
assert w3.data
860912

861913

862-
@gen_cluster(client=True)
914+
@gen_cluster(
915+
client=True,
916+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
917+
)
863918
async def test_dont_steal_already_released(c, s, a, b):
864919
future = c.submit(slowinc, 1, delay=0.05, workers=a.address)
865920
key = future.key
@@ -886,7 +941,11 @@ async def test_dont_steal_already_released(c, s, a, b):
886941
await asyncio.sleep(0.05)
887942

888943

889-
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
944+
@gen_cluster(
945+
client=True,
946+
nthreads=[("127.0.0.1", 1)] * 2,
947+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
948+
)
890949
async def test_dont_steal_long_running_tasks(c, s, a, b):
891950
def long(delay):
892951
with worker_client() as c:
@@ -974,7 +1033,7 @@ async def test_lose_task(c, s, a, b):
9741033
assert "Error" not in out
9751034

9761035

977-
@pytest.mark.parametrize("interval, expected", [(None, 100), ("500ms", 500), (2, 2)])
1036+
@pytest.mark.parametrize("interval, expected", [(None, 1000), ("500ms", 500), (2, 2)])
9781037
@gen_cluster(nthreads=[], config={"distributed.scheduler.work-stealing": False})
9791038
async def test_parse_stealing_interval(s, interval, expected):
9801039
from distributed.scheduler import WorkStealing
@@ -991,7 +1050,10 @@ async def test_parse_stealing_interval(s, interval, expected):
9911050
assert s.periodic_callbacks["stealing"].callback_time == expected
9921051

9931052

994-
@gen_cluster(client=True)
1053+
@gen_cluster(
1054+
client=True,
1055+
config={"distributed.scheduler.work-stealing-interval": "100ms"},
1056+
)
9951057
async def test_balance_with_longer_task(c, s, a, b):
9961058
np = pytest.importorskip("numpy")
9971059

0 commit comments

Comments
 (0)