/
test_scheduler.py
4369 lines (3474 loc) · 137 KB
/
test_scheduler.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
from __future__ import annotations
import asyncio
import itertools
import json
import logging
import math
import operator
import pickle
import re
import sys
from collections.abc import Collection
from itertools import product
from textwrap import dedent
from time import sleep
from unittest import mock
import cloudpickle
import psutil
import pytest
from tlz import concat, first, merge
from tornado.ioloop import IOLoop
import dask
from dask import delayed
from dask.highlevelgraph import HighLevelGraph, MaterializedLayer
from dask.utils import apply, parse_timedelta, stringify, tmpfile, typename
from distributed import (
CancelledError,
Client,
Event,
Lock,
Nanny,
SchedulerPlugin,
Worker,
fire_and_forget,
wait,
)
from distributed.comm.addressing import parse_host_port
from distributed.compatibility import LINUX, MACOS, WINDOWS, PeriodicCallback
from distributed.core import ConnectionPool, Status, clean_exception, connect, rpc
from distributed.metrics import time
from distributed.protocol import serialize
from distributed.protocol.pickle import dumps, loads
from distributed.protocol.serialize import ToPickle
from distributed.scheduler import KilledWorker, MemoryState, Scheduler, WorkerState
from distributed.utils import TimeoutError, wait_for
from distributed.utils_test import (
NO_AMM,
BlockedGatherDep,
BrokenComm,
NoSchedulerDelayWorker,
assert_story,
async_poll_for,
captured_handler,
captured_logger,
cluster,
dec,
div,
freeze_batched_send,
freeze_data_fetching,
gen_cluster,
gen_test,
inc,
nodebug,
raises_with_cause,
slowadd,
slowdec,
slowidentity,
slowinc,
tls_only_security,
varying,
wait_for_state,
)
from distributed.worker import dumps_function, dumps_task, get_worker, secede
pytestmark = pytest.mark.ci1
QUEUING_ON_BY_DEFAULT = math.isfinite(
float(dask.config.get("distributed.scheduler.worker-saturation"))
)
alice = "alice:1234"
bob = "bob:1234"
@gen_cluster()
async def test_administration(s, a, b):
assert isinstance(s.address, str)
assert s.address in str(s)
assert str(len(s.workers)) in repr(s)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
async def test_respect_data_in_memory(c, s, a):
x = delayed(inc)(1, dask_key_name="x")
y = delayed(inc)(x, dask_key_name="y")
f = c.persist(y)
await wait([f])
assert s.tasks[y.key].who_has == {s.workers[a.address]}
z = delayed(operator.add)(x, y, dask_key_name="z")
f2 = c.persist(z)
while f2.key not in s.tasks or not s.tasks[f2.key]:
assert s.tasks[y.key].who_has
await asyncio.sleep(0.0001)
@gen_cluster(client=True)
async def test_recompute_released_results(c, s, a, b):
x = delayed(inc)(1)
y = delayed(inc)(x)
yy = c.persist(y)
await wait(yy)
while s.tasks[x.key].who_has or x.key in a.data or x.key in b.data: # let x go away
await asyncio.sleep(0.01)
z = delayed(dec)(x)
zz = c.compute(z)
result = await zz
assert result == 1
@gen_cluster(client=True)
async def test_decide_worker_with_many_independent_leaves(c, s, a, b):
xs = await asyncio.gather(
c.scatter(list(range(0, 100, 2)), workers=a.address),
c.scatter(list(range(1, 100, 2)), workers=b.address),
)
xs = list(concat(zip(*xs)))
ys = [delayed(inc)(x) for x in xs]
y2s = c.persist(ys)
await wait(y2s)
nhits = sum(y.key in a.data for y in y2s[::2]) + sum(
y.key in b.data for y in y2s[1::2]
)
assert nhits > 80
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_decide_worker_with_restrictions(client, s, a, b, c):
x = client.submit(inc, 1, workers=[a.address, b.address])
await x
assert x.key in a.data or x.key in b.data
@pytest.mark.parametrize("ndeps", [0, 1, 4])
@pytest.mark.parametrize(
"nthreads",
[
[("127.0.0.1", 1)] * 5,
[("127.0.0.1", 3), ("127.0.0.1", 2), ("127.0.0.1", 1)],
],
)
def test_decide_worker_coschedule_order_neighbors(ndeps, nthreads):
@gen_cluster(
client=True,
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):
r"""
Ensure that sibling root tasks are scheduled to the same node, reducing future
data transfer.
We generate a wide layer of "root" tasks (random NumPy arrays). All of those
tasks share 0-5 trivial dependencies. The ``ndeps=0`` and ``ndeps=1`` cases are
most common in real-world use (``ndeps=1`` is basically ``da.from_array(...,
inline_array=False)`` or ``da.from_zarr``). The graph is structured like this
(though the number of tasks and workers is different):
|-W1-| |-W2-| |-W3-| |-W4-| < ---- ideal task scheduling
q r s t < --- `sum-aggregate-`
/ \ / \ / \ / \
i j k l m n o p < --- `sum-`
| | | | | | | |
a b c d e f g h < --- `random-`
\ \ \ | | / / /
TRIVIAL * 0..5
Neighboring `random-` tasks should be scheduled on the same worker. We test that
generally, only one worker holds each row of the array, that the `random-` tasks
are never transferred, and that there are few transfers overall.
"""
da = pytest.importorskip("dask.array")
np = pytest.importorskip("numpy")
if ndeps == 0:
x = da.random.random((100, 100), chunks=(10, 10))
else:
def random(**kwargs):
assert len(kwargs) == ndeps
return np.random.random((10, 10))
trivial_deps = {f"k{i}": delayed(object()) for i in range(ndeps)}
# TODO is there a simpler (non-blockwise) way to make this sort of graph?
x = da.blockwise(
random,
"yx",
new_axes={"y": (10,) * 10, "x": (10,) * 10},
dtype=float,
**trivial_deps,
)
xx, xsum = dask.persist(x, x.sum(axis=1, split_every=20))
await xsum
# Check that each chunk-row of the array is (mostly) stored on the same worker
primary_worker_key_fractions = []
secondary_worker_key_fractions = []
for keys in x.__dask_keys__():
# Iterate along rows of the array.
keys = {stringify(k) for k in keys}
# No more than 2 workers should have any keys
assert sum(any(k in w.data for k in keys) for w in workers) <= 2
# What fraction of the keys for this row does each worker hold?
key_fractions = [
len(set(w.data).intersection(keys)) / len(keys) for w in workers
]
key_fractions.sort()
# Primary worker: holds the highest percentage of keys
# Secondary worker: holds the second highest percentage of keys
primary_worker_key_fractions.append(key_fractions[-1])
secondary_worker_key_fractions.append(key_fractions[-2])
# There may be one or two rows that were poorly split across workers,
# but the vast majority of rows should only be on one worker.
assert np.mean(primary_worker_key_fractions) >= 0.9
assert np.median(primary_worker_key_fractions) == 1.0
assert np.mean(secondary_worker_key_fractions) <= 0.1
assert np.median(secondary_worker_key_fractions) == 0.0
# Check that there were few transfers
unexpected_transfers = []
for worker in workers:
for log in worker.transfer_incoming_log:
keys = log["keys"]
# The root-ish tasks should never be transferred
assert not any(k.startswith("random") for k in keys), keys
# `object-` keys (the trivial deps of the root random tasks) should be
# transferred
if any(not k.startswith("object") for k in keys):
# But not many other things should be
unexpected_transfers.append(list(keys))
# A transfer at the very end to move aggregated results is fine (necessary with
# unbalanced workers in fact), but generally there should be very very few
# transfers.
assert len(unexpected_transfers) <= 3, unexpected_transfers
test_decide_worker_coschedule_order_neighbors_()
@pytest.mark.skipif(
QUEUING_ON_BY_DEFAULT,
reason="Not relevant with queuing on; see https://github.com/dask/distributed/issues/7204",
)
@gen_cluster(
client=True,
nthreads=[("", 1)],
config={"distributed.scheduler.work-stealing": False},
)
async def test_decide_worker_rootish_while_last_worker_is_retiring(c, s, a):
"""https://github.com/dask/distributed/issues/7063"""
# Put a task in memory on the worker to be retired and prevent the other from
# acquiring a replica. This will cause a to be stuck in closing_gracefully later on,
# until we set b.block_gather_dep.
m = (await c.scatter({"m": 1}, workers=[a.address]))["m"]
evx = [Event() for _ in range(3)]
evy = Event()
async with BlockedGatherDep(s.address, nthreads=1) as b:
xs = [
c.submit(lambda ev: ev.wait(), evx[0], key="x-0", workers=[a.address]),
c.submit(lambda ev: ev.wait(), evx[1], key="x-1", workers=[a.address]),
c.submit(lambda ev: ev.wait(), evx[2], key="x-2", workers=[b.address]),
]
ys = [
c.submit(lambda x, ev: ev.wait(), xs[0], evy, key="y-0"),
c.submit(lambda x, ev: ev.wait(), xs[0], evy, key="y-1"),
c.submit(lambda x, ev: ev.wait(), xs[1], evy, key="y-2"),
c.submit(lambda x, ev: ev.wait(), xs[2], evy, key="y-3"),
c.submit(lambda x, ev: ev.wait(), xs[2], evy, key="y-4"),
c.submit(lambda x, ev: ev.wait(), xs[2], evy, key="y-5"),
]
while a.state.executing_count != 1 or b.state.executing_count != 1:
await asyncio.sleep(0.01)
# - y-2 has no restrictions
# - TaskGroup(y) has more than 4 tasks (total_nthreads * 2)
# - TaskGroup(y) has less than 5 dependency groups
# - TaskGroup(y) has less than 5 dependency tasks
assert s.is_rootish(s.tasks["y-2"])
await evx[0].set()
await wait_for_state("y-0", "processing", s)
await wait_for_state("y-1", "processing", s)
assert s.tasks["y-2"].group.last_worker == s.workers[a.address]
assert s.tasks["y-2"].group.last_worker_tasks_left == 1
# Take a out of the running pool, but without removing it from the cluster
# completely
retire_task = asyncio.create_task(c.retire_workers([a.address]))
# Wait until AMM sends AcquireReplicasEvent to b to move away m
await b.in_gather_dep.wait()
assert s.workers[a.address].status == Status.closing_gracefully
# Transition y-2 to processing. Normally, it would be scheduled on a, but it's
# not a running worker, so we must choose b
await evx[1].set()
await wait_for_state("y-2", "processing", s)
await wait_for_state("y-2", "waiting", b) # x-1 is in memory on a
# Cleanup
b.block_gather_dep.set()
await evx[2].set()
await evy.set()
await retire_task
await wait(xs + ys)
@pytest.mark.slow
@gen_cluster(
nthreads=[("", 2)] * 4,
client=True,
config={"distributed.scheduler.worker-saturation": 1.0},
)
async def test_graph_execution_width(c, s, *workers):
"""
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 = 0
lock = dask.utils.SerializableLock()
log = []
def __init__(self):
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)]
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)
# 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
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_forget_tasks_while_processing(c, s, a, b):
events = [Event() for _ in range(10)]
futures = c.map(Event.wait, events)
await events[0].set()
await futures[0]
await c.close()
assert not s.tasks
@pytest.mark.slow
@gen_cluster(client=True, nthreads=[("", 1)], Worker=Nanny)
async def test_restart_while_processing(c, s, a, b):
events = [Event() for _ in range(10)]
futures = c.map(Event.wait, events)
await events[0].set()
await futures[0]
# TODO slow because worker waits a while for the task to finish
await c.restart()
assert not s.tasks
@gen_cluster(
client=True,
nthreads=[("", 1)] * 3,
config={"distributed.scheduler.worker-saturation": 1.0},
)
async def test_queued_release_multiple_workers(c, s, *workers):
async with Client(s.address, asynchronous=True) as c2:
event = Event(client=c2)
rootish_threshold = s.total_nthreads * 2 + 1
first_batch = c.map(
lambda i: event.wait(),
range(rootish_threshold),
key=[f"first-{i}" for i in range(rootish_threshold)],
)
await async_poll_for(lambda: s.queued, 5)
second_batch = c2.map(
lambda i: event.wait(),
range(rootish_threshold),
key=[f"second-{i}" for i in range(rootish_threshold)],
fifo_timeout=0,
)
await async_poll_for(lambda: second_batch[0].key in s.tasks, 5)
# All of the second batch should be queued after the first batch
assert [ts.key for ts in s.queued.sorted()] == [
f.key
for f in itertools.chain(first_batch[s.total_nthreads :], second_batch)
]
# Cancel the first batch.
# Use `Client.close` instead of `del first_batch` because deleting futures sends cancellation
# messages one at a time. We're testing here that when multiple workers have open slots, we don't
# recommend the same queued tasks for every worker, so we need a bulk cancellation operation.
await c.close()
del c, first_batch
await async_poll_for(lambda: len(s.tasks) == len(second_batch), 5)
# Second batch should move up the queue and start processing
assert len(s.queued) == len(second_batch) - s.total_nthreads, list(
s.queued.sorted()
)
await event.set()
await c2.gather(second_batch)
@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,
"distributed.scheduler.worker-saturation": 1.0,
},
)
async def test_queued_paused_new_worker(c, s, a, b):
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.idle
assert not s.idle_task_count
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)
del final
while s.tasks:
await asyncio.sleep(0.01)
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
assert not s.idle_task_count
# un-pause
a.status = Status.running
b.status = Status.running
while not s.running:
await asyncio.sleep(0.01)
if queue:
assert not s.idle # workers should have been (or already were) filled
# If queuing is disabled, all workers might already be saturated when they un-pause.
assert not s.idle_task_count
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_poll_for(lambda: len(s.queued) == 6, timeout=5)
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_poll_for(lambda: len(s.queued) == 6, timeout=5)
await event.set()
await wait(fs)
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_secede_opens_slot(c, s, a):
first = Event()
second = Event()
def func(first, second):
first.wait()
secede()
second.wait()
fs = c.map(func, [first] * 5, [second] * 5)
await async_poll_for(lambda: a.state.executing, timeout=5)
await first.set()
await async_poll_for(lambda: len(a.state.long_running) == len(fs), timeout=5)
await second.set()
await c.gather(fs)
@pytest.mark.parametrize(
"saturation_config, expected_task_counts",
[
(2.5, (5, 3)),
(2.0, (4, 2)),
(1.1, (3, 2)),
(1.0, (2, 1)),
(0.1, (1, 1)),
# This is necessary because there's no way to parse a float infinite from
# a DASK_* environment variable
("inf", (6, 4)),
(float("inf"), (6, 4)),
# ^ depends on root task assignment logic; ok if changes, just needs to add up to 10
],
)
def test_saturation_factor(
saturation_config: int | float | str, expected_task_counts: tuple[int, int]
) -> None:
@gen_cluster(
client=True,
nthreads=[("", 2), ("", 1)],
config={
"distributed.scheduler.worker-saturation": saturation_config,
},
)
async def _test_saturation_factor(c, s, a, b):
saturation = float(saturation_config)
event = Event()
fs = c.map(
lambda _: event.wait(), range(10), key=[f"wait-{i}" for i in range(10)]
)
while a.state.executing_count < min(
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)
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)
_test_saturation_factor()
@gen_test()
async def test_bad_saturation_factor():
with pytest.raises(ValueError, match="foo"):
with dask.config.set({"distributed.scheduler.worker-saturation": "foo"}):
async with Scheduler(dashboard_address=":0"):
pass
@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)
y = client.submit(inc, x, workers=[a.address, b.address])
await wait(y)
assert y.key in a.data or y.key in b.data
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_balance_with_restrictions(client, s, a, b, c):
[x], [y] = await asyncio.gather(
client.scatter([[1, 2, 3]], workers=a.address),
client.scatter([1], workers=c.address),
)
z = client.submit(inc, 1, workers=[a.address, c.address])
await wait(z)
assert s.tasks[z.key].who_has == {s.workers[c.address]}
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_no_valid_workers(client, s, a, b, c):
x = client.submit(inc, 1, workers="127.0.0.5:9999")
while not s.tasks:
await asyncio.sleep(0.01)
assert s.tasks[x.key] in s.unrunnable
with pytest.raises(TimeoutError):
await wait_for(x, 0.05)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_no_valid_workers_loose_restrictions(client, s, a, b, c):
x = client.submit(inc, 1, workers="127.0.0.5:9999", allow_other_workers=True)
result = await x
assert result == 2
@pytest.mark.parametrize("queue", [False, True])
@gen_cluster(client=True, nthreads=[])
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)
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 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):
await s.retire_workers(workers=[])
@gen_cluster()
async def test_server_listens_to_other_ops(s, a, b):
async with rpc(s.address) as r:
ident = await r.identity()
assert ident["type"] == "Scheduler"
assert ident["id"].lower().startswith("scheduler")
@gen_cluster(client=True)
async def test_remove_worker_from_scheduler(c, s, a, b):
"""see also test_ready_remove_worker"""
ev = Event()
futs = c.map(lambda x, ev: ev.wait(), range(20), ev=ev)
while len(s.tasks) != len(futs):
await asyncio.sleep(0.01)
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(s.queued) == len(futs)
await ev.set()
await c.gather(futs)
@gen_cluster()
async def test_remove_worker_by_name_from_scheduler(s, a, b):
assert a.address in s.stream_comms
assert await s.remove_worker(address=a.name, stimulus_id="test") == "OK"
assert a.address not in s.workers
assert (
await s.remove_worker(address=a.address, stimulus_id="test")
== "already-removed"
)
@gen_cluster(config={"distributed.scheduler.events-cleanup-delay": "10 ms"})
async def test_clear_events_worker_removal(s, a, b):
assert a.address in s.events
assert a.address in s.workers
assert b.address in s.events
assert b.address in s.workers
await s.remove_worker(address=a.address, stimulus_id="test")
# Shortly after removal, the events should still be there
assert a.address in s.events
assert a.address not in s.workers
s.validate_state()
start = time()
while a.address in s.events:
await asyncio.sleep(0.01)
assert time() < start + 2
assert b.address in s.events
@gen_cluster(
config={"distributed.scheduler.events-cleanup-delay": "10 ms"}, client=True
)
async def test_clear_events_client_removal(c, s, a, b):
assert c.id in s.events
s.remove_client(c.id)
assert c.id in s.events
assert c.id not in s.clients
assert c not in s.clients
s.remove_client(c.id)
# If it doesn't reconnect after a given time, the events log should be cleared
start = time()
while c.id in s.events:
await asyncio.sleep(0.01)
assert time() < start + 2
@gen_cluster(client=True, nthreads=[])
async def test_add_worker(c, s):
x = c.submit(inc, 1, key="x")
await wait_for_state("x", ("queued", "no-worker"), s)
s.validate_state()
async with Worker(s.address) as w:
s.validate_state()
assert w.ip in s.host_info
assert s.host_info[w.ip]["addresses"] == {w.address}
assert await x == 2
@gen_cluster(scheduler_kwargs={"blocked_handlers": ["feed"]})
async def test_blocked_handlers_are_respected(s, a, b):
def func(scheduler):
return dumps(dict(scheduler.worker_info))
comm = await connect(s.address)
await comm.write({"op": "feed", "function": dumps(func), "interval": 0.01})
response = await comm.read()
_, exc, _ = clean_exception(response["exception"], response["traceback"])
assert isinstance(exc, ValueError)
assert "'feed' handler has been explicitly disallowed" in repr(exc)
await comm.close()
@gen_cluster(
nthreads=[], config={"distributed.scheduler.blocked-handlers": ["test-handler"]}
)
async def test_scheduler_init_pulls_blocked_handlers_from_config(s):
assert s.blocked_handlers == ["test-handler"]
@gen_cluster()
async def test_feed(s, a, b):
def func(scheduler):
return dumps({addr: ws.clean() for addr, ws in scheduler.workers.items()})
comm = await connect(s.address)
await comm.write({"op": "feed", "function": dumps(func), "interval": 0.01})
for _ in range(5):
response = await comm.read()
expected = {addr: ws.clean() for addr, ws in s.workers.items()}
assert cloudpickle.loads(response) == expected
await comm.close()
@gen_cluster()
async def test_feed_setup_teardown(s, a, b):
def setup(scheduler):
return 1
def func(scheduler, state):
assert state == 1
return "OK"
def teardown(scheduler, state):
scheduler.flag = "done"
comm = await connect(s.address)
await comm.write(
{
"op": "feed",
"function": dumps(func),
"setup": dumps(setup),
"teardown": dumps(teardown),
"interval": 0.01,
}
)
for _ in range(5):
response = await comm.read()
assert response == "OK"
await comm.close()
start = time()
while not hasattr(s, "flag"):
await asyncio.sleep(0.01)
assert time() - start < 5
@gen_cluster()
async def test_feed_large_bytestring(s, a, b):
np = pytest.importorskip("numpy")
x = np.ones(10000000)
def func(scheduler):
y = x
return True
comm = await connect(s.address)
await comm.write({"op": "feed", "function": dumps(func), "interval": 0.05})
for _ in range(5):
response = await comm.read()
assert response is True
await comm.close()
@gen_cluster(client=True)
async def test_delete_data(c, s, a, b):
d = await c.scatter({"x": 1, "y": 2, "z": 3})
assert {ts.key for ts in s.tasks.values() if ts.who_has} == {"x", "y", "z"}
assert set(a.data) | set(b.data) == {"x", "y", "z"}
assert merge(a.data, b.data) == {"x": 1, "y": 2, "z": 3}
del d["x"]
del d["y"]
start = time()
while set(a.data) | set(b.data) != {"z"}:
await asyncio.sleep(0.01)
assert time() < start + 5
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
async def test_delete(c, s, a):
x = c.submit(inc, 1)
await x
assert x.key in s.tasks
assert x.key in a.data
await c._cancel(x)
start = time()
while x.key in a.data:
await asyncio.sleep(0.01)
assert time() < start + 5
assert x.key not in s.tasks
s.report_on_key(key=x.key)
def test_dumps_function():
a = dumps_function(inc)
assert cloudpickle.loads(a)(10) == 11
b = dumps_function(inc)
assert a is b
c = dumps_function(dec)
assert a != c
def test_dumps_task():
d = dumps_task((inc, 1))
assert set(d) == {"function", "args"}
def f(x, y=2):
return x + y
d = dumps_task((apply, f, (1,), {"y": 10}))
assert cloudpickle.loads(d["function"])(1, 2) == 3
assert cloudpickle.loads(d["args"]) == (1,)
assert cloudpickle.loads(d["kwargs"]) == {"y": 10}
d = dumps_task((apply, f, (1,)))
assert cloudpickle.loads(d["function"])(1, 2) == 3
assert cloudpickle.loads(d["args"]) == (1,)
assert set(d) == {"function", "args"}
@pytest.mark.parametrize("worker_saturation", [1.0, float("inf")])
@gen_cluster(client=True)
async def test_ready_remove_worker(c, s, a, b, worker_saturation):
"""see also test_remove_worker_from_scheduler"""
s.WORKER_SATURATION = worker_saturation
ev = Event()
futs = c.map(lambda x, ev: ev.wait(), range(20), ev=ev)
while len(s.tasks) != len(futs):
await asyncio.sleep(0.01)
if s.WORKER_SATURATION == 1:
cmp = operator.eq
elif math.isinf(s.WORKER_SATURATION):
cmp = operator.gt
else:
pytest.fail(f"{s.WORKER_SATURATION=}, 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
)
await s.remove_worker(address=a.address, stimulus_id="test")
assert set(s.workers) == {b.address}
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
)
await ev.set()
@pytest.mark.slow
@gen_cluster(client=True, Worker=Nanny, timeout=60)