/
test_client.py
7062 lines (5233 loc) · 192 KB
/
test_client.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
import asyncio
import functools
import gc
import inspect
import logging
import os
import pickle
import random
import subprocess
import sys
import threading
import traceback
import warnings
import weakref
import zipfile
from collections import deque
from contextlib import suppress
from functools import partial
from operator import add
from threading import Semaphore
from time import sleep
import psutil
import pytest
from tlz import concat, first, identity, isdistinct, merge, pluck, valmap
import dask
import dask.bag as db
from dask import delayed
from dask.optimization import SubgraphCallable
from dask.utils import stringify, tmpfile
from distributed import (
CancelledError,
Executor,
LocalCluster,
Nanny,
TimeoutError,
Worker,
fire_and_forget,
get_client,
get_worker,
performance_report,
profile,
secede,
)
from distributed.client import (
Client,
Future,
_get_global_client,
as_completed,
default_client,
futures_of,
get_task_metadata,
temp_default_client,
tokenize,
wait,
)
from distributed.comm import CommClosedError
from distributed.compatibility import LINUX, WINDOWS
from distributed.core import Status
from distributed.metrics import time
from distributed.objects import HasWhat, WhoHas
from distributed.scheduler import (
COMPILED,
CollectTaskMetaDataPlugin,
KilledWorker,
Scheduler,
)
from distributed.sizeof import sizeof
from distributed.utils import is_valid_xml, mp_context, sync, tmp_text
from distributed.utils_test import (
TaskStateMetadataPlugin,
_UnhashableCallable,
async_wait_for,
asyncinc,
captured_logger,
cluster,
dec,
div,
double,
gen_cluster,
gen_test,
geninc,
inc,
map_varying,
nodebug,
popen,
pristine_loop,
randominc,
save_sys_modules,
slowadd,
slowdec,
slowinc,
throws,
varying,
wait_for,
)
pytestmark = pytest.mark.ci1
@gen_cluster(client=True)
async def test_submit(c, s, a, b):
x = c.submit(inc, 10)
assert not x.done()
assert isinstance(x, Future)
assert x.client is c
result = await x
assert result == 11
assert x.done()
y = c.submit(inc, 20)
z = c.submit(add, x, y)
result = await z
assert result == 11 + 21
s.validate_state()
@gen_cluster(client=True)
async def test_map(c, s, a, b):
L1 = c.map(inc, range(5))
assert len(L1) == 5
assert isdistinct(x.key for x in L1)
assert all(isinstance(x, Future) for x in L1)
result = await L1[0]
assert result == inc(0)
assert len(s.tasks) == 5
L2 = c.map(inc, L1)
result = await L2[1]
assert result == inc(inc(1))
assert len(s.tasks) == 10
# assert L1[0].key in s.tasks[L2[0].key]
total = c.submit(sum, L2)
result = await total
assert result == sum(map(inc, map(inc, range(5))))
L3 = c.map(add, L1, L2)
result = await L3[1]
assert result == inc(1) + inc(inc(1))
L4 = c.map(add, range(3), range(4))
results = await c.gather(L4)
assert results == list(map(add, range(3), range(4)))
def f(x, y=10):
return x + y
L5 = c.map(f, range(5), y=5)
results = await c.gather(L5)
assert results == list(range(5, 10))
y = c.submit(f, 10)
L6 = c.map(f, range(5), y=y)
results = await c.gather(L6)
assert results == list(range(20, 25))
s.validate_state()
@gen_cluster(client=True)
async def test_map_empty(c, s, a, b):
L1 = c.map(inc, [], pure=False)
assert len(L1) == 0
results = await c.gather(L1)
assert results == []
@gen_cluster(client=True)
async def test_map_keynames(c, s, a, b):
futures = c.map(inc, range(4), key="INC")
assert all(f.key.startswith("INC") for f in futures)
assert isdistinct(f.key for f in futures)
futures2 = c.map(inc, [5, 6, 7, 8], key="INC")
assert [f.key for f in futures] != [f.key for f in futures2]
keys = ["inc-1", "inc-2", "inc-3", "inc-4"]
futures = c.map(inc, range(4), key=keys)
assert [f.key for f in futures] == keys
@gen_cluster(client=True)
async def test_map_retries(c, s, a, b):
args = [
[ZeroDivisionError("one"), 2, 3],
[4, 5, 6],
[ZeroDivisionError("seven"), ZeroDivisionError("eight"), 9],
]
x, y, z = c.map(*map_varying(args), retries=2)
assert await x == 2
assert await y == 4
assert await z == 9
x, y, z = c.map(*map_varying(args), retries=1, pure=False)
assert await x == 2
assert await y == 4
with pytest.raises(ZeroDivisionError, match="eight"):
await z
x, y, z = c.map(*map_varying(args), retries=0, pure=False)
with pytest.raises(ZeroDivisionError, match="one"):
await x
assert await y == 4
with pytest.raises(ZeroDivisionError, match="seven"):
await z
@gen_cluster(client=True)
async def test_map_batch_size(c, s, a, b):
result = c.map(inc, range(100), batch_size=10)
result = await c.gather(result)
assert result == list(range(1, 101))
result = c.map(add, range(100), range(100), batch_size=10)
result = await c.gather(result)
assert result == list(range(0, 200, 2))
# mismatch shape
result = c.map(add, range(100, 200), range(10), batch_size=2)
result = await c.gather(result)
assert result == list(range(100, 120, 2))
@gen_cluster(client=True)
async def test_custom_key_with_batches(c, s, a, b):
"""Test of <https://github.com/dask/distributed/issues/4588>"""
futs = c.map(
lambda x: x ** 2,
range(10),
batch_size=5,
key=[str(x) for x in range(10)],
)
assert len(futs) == 10
await wait(futs)
@gen_cluster(client=True)
async def test_compute_retries(c, s, a, b):
args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3]
# Sanity check for varying() use
x = c.compute(delayed(varying(args))())
with pytest.raises(ZeroDivisionError, match="one"):
await x
# Same retries for all
x = c.compute(delayed(varying(args))(), retries=1)
with pytest.raises(ZeroDivisionError, match="two"):
await x
x = c.compute(delayed(varying(args))(), retries=2)
assert await x == 3
args.append(4)
x = c.compute(delayed(varying(args))(), retries=2)
assert await x == 3
@gen_cluster(client=True)
async def test_compute_retries_annotations(c, s, a, b):
# Per-future retries
xargs = [ZeroDivisionError("one"), ZeroDivisionError("two"), 30, 40]
yargs = [ZeroDivisionError("five"), ZeroDivisionError("six"), 70]
zargs = [80, 90, 100]
with dask.annotate(retries=2):
x = delayed(varying(xargs))()
y = delayed(varying(yargs))()
x, y = c.compute([x, y], optimize_graph=False)
gc.collect()
assert await x == 30
with pytest.raises(ZeroDivisionError, match="five"):
await y
x = delayed(varying(xargs))()
with dask.annotate(retries=2):
y = delayed(varying(yargs))()
z = delayed(varying(zargs))()
x, y, z = c.compute([x, y, z], optimize_graph=False)
with pytest.raises(ZeroDivisionError, match="one"):
await x
assert await y == 70
assert await z == 80
def test_retries_get(c):
args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3]
x = delayed(varying(args))()
assert x.compute(retries=5) == 3
args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3]
x = delayed(varying(args))()
with pytest.raises(ZeroDivisionError):
x.compute()
@gen_cluster(client=True)
async def test_compute_persisted_retries(c, s, a, b):
args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3]
# Sanity check
x = c.persist(delayed(varying(args))())
fut = c.compute(x)
with pytest.raises(ZeroDivisionError, match="one"):
await fut
x = c.persist(delayed(varying(args))())
fut = c.compute(x, retries=1)
with pytest.raises(ZeroDivisionError, match="two"):
await fut
x = c.persist(delayed(varying(args))())
fut = c.compute(x, retries=2)
assert await fut == 3
args.append(4)
x = c.persist(delayed(varying(args))())
fut = c.compute(x, retries=3)
assert await fut == 3
@gen_cluster(client=True)
async def test_persist_retries(c, s, a, b):
# Same retries for all
args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 3]
x = c.persist(delayed(varying(args))(), retries=1)
x = c.compute(x)
with pytest.raises(ZeroDivisionError, match="two"):
await x
x = c.persist(delayed(varying(args))(), retries=2)
x = c.compute(x)
assert await x == 3
@gen_cluster(client=True)
async def test_persist_retries_annotations(c, s, a, b):
# Per-key retries
xargs = [ZeroDivisionError("one"), ZeroDivisionError("two"), 30, 40]
yargs = [ZeroDivisionError("five"), ZeroDivisionError("six"), 70]
zargs = [80, 90, 100]
x = delayed(varying(xargs))()
with dask.annotate(retries=2):
y = delayed(varying(yargs))()
z = delayed(varying(zargs))()
x, y, z = c.persist([x, y, z], optimize_graph=False)
x, y, z = c.compute([x, y, z])
with pytest.raises(ZeroDivisionError, match="one"):
await x
assert await y == 70
assert await z == 80
@gen_cluster(client=True)
async def test_retries_dask_array(c, s, a, b):
da = pytest.importorskip("dask.array")
x = da.ones((10, 10), chunks=(3, 3))
future = c.compute(x.sum(), retries=2)
y = await future
assert y == 100
@gen_cluster(client=True)
async def test_future_repr(c, s, a, b):
pd = pytest.importorskip("pandas")
x = c.submit(inc, 10)
y = c.submit(pd.DataFrame, {"x": [1, 2, 3]})
await x
await y
for func in [repr, lambda x: x._repr_html_()]:
assert str(x.key) in func(x)
assert str(x.status) in func(x)
assert str(x.status) in repr(c.futures[x.key])
assert "int" in func(x)
assert "pandas" in func(y)
assert "DataFrame" in func(y)
@gen_cluster(client=True)
async def test_future_tuple_repr(c, s, a, b):
da = pytest.importorskip("dask.array")
y = da.arange(10, chunks=(5,)).persist()
f = futures_of(y)[0]
for func in [repr, lambda x: x._repr_html_()]:
for k in f.key:
assert str(k) in func(f)
@gen_cluster(client=True)
async def test_Future_exception(c, s, a, b):
x = c.submit(div, 1, 0)
result = await x.exception()
assert isinstance(result, ZeroDivisionError)
x = c.submit(div, 1, 1)
result = await x.exception()
assert result is None
def test_Future_exception_sync(c):
x = c.submit(div, 1, 0)
assert isinstance(x.exception(), ZeroDivisionError)
x = c.submit(div, 1, 1)
assert x.exception() is None
@gen_cluster(client=True)
async def test_Future_release(c, s, a, b):
# Released Futures should be removed timely from the Client
x = c.submit(div, 1, 1)
await x
x.release()
await asyncio.sleep(0)
assert not c.futures
x = c.submit(slowinc, 1, delay=0.5)
x.release()
await asyncio.sleep(0)
assert not c.futures
x = c.submit(div, 1, 0)
await x.exception()
x.release()
await asyncio.sleep(0)
assert not c.futures
def test_Future_release_sync(c):
# Released Futures should be removed timely from the Client
x = c.submit(div, 1, 1)
x.result()
x.release()
wait_for(lambda: not c.futures, timeout=0.3)
x = c.submit(slowinc, 1, delay=0.8)
x.release()
wait_for(lambda: not c.futures, timeout=0.3)
x = c.submit(div, 1, 0)
x.exception()
x.release()
wait_for(lambda: not c.futures, timeout=0.3)
def test_short_tracebacks(loop, c):
tblib = pytest.importorskip("tblib")
future = c.submit(div, 1, 0)
try:
future.result()
except Exception:
_, _, tb = sys.exc_info()
tb = tblib.Traceback(tb).to_dict()
n = 0
while tb is not None:
n += 1
tb = tb["tb_next"]
assert n < 5
@gen_cluster(client=True)
async def test_map_naming(c, s, a, b):
L1 = c.map(inc, range(5))
L2 = c.map(inc, range(5))
assert [x.key for x in L1] == [x.key for x in L2]
L3 = c.map(inc, [1, 1, 1, 1])
assert len({x._state for x in L3}) == 1
L4 = c.map(inc, [1, 1, 1, 1], pure=False)
assert len({x._state for x in L4}) == 4
@gen_cluster(client=True)
async def test_submit_naming(c, s, a, b):
a = c.submit(inc, 1)
b = c.submit(inc, 1)
assert a._state is b._state
c = c.submit(inc, 1, pure=False)
assert c.key != a.key
@gen_cluster(client=True)
async def test_exceptions(c, s, a, b):
x = c.submit(div, 1, 2)
result = await x
assert result == 1 / 2
x = c.submit(div, 1, 0)
with pytest.raises(ZeroDivisionError):
await x
x = c.submit(div, 10, 2) # continues to operate
result = await x
assert result == 10 / 2
@gen_cluster()
async def test_gc(s, a, b):
c = await Client(s.address, asynchronous=True)
x = c.submit(inc, 10)
await x
assert s.tasks[x.key].who_has
x.__del__()
await async_wait_for(
lambda: x.key not in s.tasks or not s.tasks[x.key].who_has, timeout=0.3
)
await c.close()
def test_thread(c):
x = c.submit(inc, 1)
assert x.result() == 2
x = c.submit(slowinc, 1, delay=0.3)
with pytest.raises(TimeoutError):
x.result(timeout="10 ms")
assert x.result() == 2
def test_sync_exceptions(c):
x = c.submit(div, 10, 2)
assert x.result() == 5
y = c.submit(div, 10, 0)
try:
y.result()
assert False
except ZeroDivisionError:
pass
z = c.submit(div, 10, 5)
assert z.result() == 2
@gen_cluster(client=True)
async def test_gather(c, s, a, b):
x = c.submit(inc, 10)
y = c.submit(inc, x)
result = await c.gather(x)
assert result == 11
result = await c.gather([x])
assert result == [11]
result = await c.gather({"x": x, "y": [y]})
assert result == {"x": 11, "y": [12]}
@gen_cluster(client=True)
async def test_gather_lost(c, s, a, b):
[x] = await c.scatter([1], workers=a.address)
y = c.submit(inc, 1, workers=b.address)
await a.close()
with pytest.raises(Exception):
await c.gather([x, y])
def test_gather_sync(c):
x = c.submit(inc, 1)
assert c.gather(x) == 2
y = c.submit(div, 1, 0)
with pytest.raises(ZeroDivisionError):
c.gather([x, y])
[xx] = c.gather([x, y], errors="skip")
assert xx == 2
@gen_cluster(client=True)
async def test_gather_strict(c, s, a, b):
x = c.submit(div, 2, 1)
y = c.submit(div, 1, 0)
with pytest.raises(ZeroDivisionError):
await c.gather([x, y])
[xx] = await c.gather([x, y], errors="skip")
assert xx == 2
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
async def test_gather_skip(c, s, a):
x = c.submit(div, 1, 0, priority=10)
y = c.submit(slowinc, 1, delay=0.5)
with captured_logger(logging.getLogger("distributed.scheduler")) as sched:
with captured_logger(logging.getLogger("distributed.client")) as client:
L = await c.gather([x, y], errors="skip")
assert L == [2]
assert not client.getvalue()
assert not sched.getvalue()
@gen_cluster(client=True)
async def test_limit_concurrent_gathering(c, s, a, b):
futures = c.map(inc, range(100))
await c.gather(futures)
assert len(a.outgoing_transfer_log) + len(b.outgoing_transfer_log) < 100
@gen_cluster(client=True)
async def test_get(c, s, a, b):
future = c.get({"x": (inc, 1)}, "x", sync=False)
assert isinstance(future, Future)
result = await future
assert result == 2
futures = c.get({"x": (inc, 1)}, ["x"], sync=False)
assert isinstance(futures[0], Future)
result = await c.gather(futures)
assert result == [2]
futures = c.get({}, [], sync=False)
result = await c.gather(futures)
assert result == []
result = await c.get(
{("x", 1): (inc, 1), ("x", 2): (inc, ("x", 1))}, ("x", 2), sync=False
)
assert result == 3
def test_get_sync(c):
assert c.get({"x": (inc, 1)}, "x") == 2
def test_no_future_references(c):
from weakref import WeakSet
ws = WeakSet()
futures = c.map(inc, range(10))
ws.update(futures)
del futures
import gc
gc.collect()
start = time()
while list(ws):
sleep(0.01)
assert time() < start + 30
def test_get_sync_optimize_graph_passes_through(c):
bag = db.range(10, npartitions=3).map(inc)
dask.compute(bag.sum(), optimize_graph=False)
@gen_cluster(client=True)
async def test_gather_errors(c, s, a, b):
def f(a, b):
raise TypeError
def g(a, b):
raise AttributeError
future_f = c.submit(f, 1, 2)
future_g = c.submit(g, 1, 2)
with pytest.raises(TypeError):
await c.gather(future_f)
with pytest.raises(AttributeError):
await c.gather(future_g)
await a.close()
@gen_cluster(client=True)
async def test_wait(c, s, a, b):
x = c.submit(inc, 1)
y = c.submit(inc, 1)
z = c.submit(inc, 2)
done, not_done = await wait([x, y, z])
assert done == {x, y, z}
assert not_done == set()
assert x.status == y.status == "finished"
@gen_cluster(client=True)
async def test_wait_first_completed(c, s, a, b):
x = c.submit(slowinc, 1)
y = c.submit(slowinc, 1)
z = c.submit(inc, 2)
done, not_done = await wait([x, y, z], return_when="FIRST_COMPLETED")
assert done == {z}
assert not_done == {x, y}
assert z.status == "finished"
assert x.status == "pending"
assert y.status == "pending"
@gen_cluster(client=True)
async def test_wait_timeout(c, s, a, b):
future = c.submit(sleep, 0.3)
with pytest.raises(TimeoutError):
await wait(future, timeout=0.01)
def test_wait_sync(c):
x = c.submit(inc, 1)
y = c.submit(inc, 2)
done, not_done = wait([x, y])
assert done == {x, y}
assert not_done == set()
assert x.status == y.status == "finished"
future = c.submit(sleep, 0.3)
with pytest.raises(TimeoutError):
wait(future, timeout=0.01)
def test_wait_informative_error_for_timeouts(c):
x = c.submit(inc, 1)
y = c.submit(inc, 2)
try:
wait(x, y)
except Exception as e:
assert "timeout" in str(e)
assert "list" in str(e)
@gen_cluster(client=True)
async def test_garbage_collection(c, s, a, b):
x = c.submit(inc, 1)
y = c.submit(inc, 1)
assert c.refcount[x.key] == 2
x.__del__()
await asyncio.sleep(0)
assert c.refcount[x.key] == 1
z = c.submit(inc, y)
y.__del__()
await asyncio.sleep(0)
result = await z
assert result == 3
ykey = y.key
y.__del__()
await asyncio.sleep(0)
assert ykey not in c.futures
@gen_cluster(client=True)
async def test_garbage_collection_with_scatter(c, s, a, b):
[future] = await c.scatter([1])
assert future.key in c.futures
assert future.status == "finished"
assert s.who_wants[future.key] == {c.id}
key = future.key
assert c.refcount[key] == 1
future.__del__()
await asyncio.sleep(0)
assert c.refcount[key] == 0
while key in s.tasks and s.tasks[key].who_has:
await asyncio.sleep(0.1)
@gen_cluster(client=True)
async def test_recompute_released_key(c, s, a, b):
x = c.submit(inc, 100)
result1 = await x
xkey = x.key
del x
import gc
gc.collect()
await asyncio.sleep(0)
assert c.refcount[xkey] == 0
# 1 second batching needs a second action to trigger
while xkey in s.tasks and s.tasks[xkey].who_has or xkey in a.data or xkey in b.data:
await asyncio.sleep(0.1)
x = c.submit(inc, 100)
assert x.key in c.futures
result2 = await x
assert result1 == result2
@pytest.mark.slow
@gen_cluster(client=True)
async def test_long_tasks_dont_trigger_timeout(c, s, a, b):
from time import sleep
x = c.submit(sleep, 3)
await x
@pytest.mark.skip
@gen_cluster(client=True)
async def test_missing_data_heals(c, s, a, b):
a.validate = False
b.validate = False
x = c.submit(inc, 1)
y = c.submit(inc, x)
z = c.submit(inc, y)
await wait([x, y, z])
# Secretly delete y's key
if y.key in a.data:
del a.data[y.key]
a.release_key(y.key)
if y.key in b.data:
del b.data[y.key]
b.release_key(y.key)
await asyncio.sleep(0)
w = c.submit(add, y, z)
result = await w
assert result == 3 + 4
@pytest.mark.skip
@gen_cluster(client=True)
async def test_gather_robust_to_missing_data(c, s, a, b):
a.validate = False
b.validate = False
x, y, z = c.map(inc, range(3))
await wait([x, y, z]) # everything computed
for f in [x, y]:
for w in [a, b]:
if f.key in w.data:
del w.data[f.key]
await asyncio.sleep(0)
w.release_key(f.key)
xx, yy, zz = await c.gather([x, y, z])
assert (xx, yy, zz) == (1, 2, 3)
@pytest.mark.skip
@gen_cluster(client=True)
async def test_gather_robust_to_nested_missing_data(c, s, a, b):
a.validate = False
b.validate = False
w = c.submit(inc, 1)
x = c.submit(inc, w)
y = c.submit(inc, x)
z = c.submit(inc, y)
await wait([z])
for worker in [a, b]:
for datum in [y, z]:
if datum.key in worker.data:
del worker.data[datum.key]
await asyncio.sleep(0)
worker.release_key(datum.key)
result = await c.gather([z])
assert result == [inc(inc(inc(inc(1))))]
@gen_cluster(client=True)
async def test_tokenize_on_futures(c, s, a, b):
x = c.submit(inc, 1)
y = c.submit(inc, 1)
tok = tokenize(x)
assert tokenize(x) == tokenize(x)
assert tokenize(x) == tokenize(y)
c.futures[x.key].finish()
assert tok == tokenize(y)
@pytest.mark.skipif(not LINUX, reason="Need 127.0.0.2 to mean localhost")
@gen_cluster([("127.0.0.1", 1), ("127.0.0.2", 2)], client=True)
async def test_restrictions_submit(c, s, a, b):
x = c.submit(inc, 1, workers={a.ip})
y = c.submit(inc, x, workers={b.ip})
await wait([x, y])
assert s.host_restrictions[x.key] == {a.ip}
assert x.key in a.data
assert s.host_restrictions[y.key] == {b.ip}
assert y.key in b.data
@gen_cluster(client=True)
async def test_restrictions_ip_port(c, s, a, b):
x = c.submit(inc, 1, workers={a.address})
y = c.submit(inc, x, workers={b.address})
await wait([x, y])
assert s.worker_restrictions[x.key] == {a.address}
assert x.key in a.data
assert s.worker_restrictions[y.key] == {b.address}
assert y.key in b.data
@pytest.mark.skipif(not LINUX, reason="Need 127.0.0.2 to mean localhost")
@gen_cluster([("127.0.0.1", 1), ("127.0.0.2", 2)], client=True)
async def test_restrictions_map(c, s, a, b):
L = c.map(inc, range(5), workers={a.ip})
await wait(L)
assert set(a.data) == {x.key for x in L}
assert not b.data
for x in L:
assert s.host_restrictions[x.key] == {a.ip}
@pytest.mark.skipif(not LINUX, reason="Need 127.0.0.2 to mean localhost")
@gen_cluster([("127.0.0.1", 1), ("127.0.0.2", 2)], client=True)
async def test_restrictions_get(c, s, a, b):
dsk = {"x": 1, "y": (inc, "x"), "z": (inc, "y")}
futures = c.get(dsk, ["y", "z"], workers=a.ip, sync=False)
result = await c.gather(futures)
assert result == [2, 3]
assert "y" in a.data
assert "z" in a.data
assert len(b.data) == 0
@gen_cluster(client=True)
async def test_restrictions_get_annotate(c, s, a, b):
x = 1
with dask.annotate(workers=a.address):
y = delayed(inc)(x)
with dask.annotate(workers=b.address):
z = delayed(inc)(y)
futures = c.get(z.__dask_graph__(), [y.key, z.key], sync=False)
result = await c.gather(futures)
assert result == [2, 3]
assert y.key in a.data
assert z.key in b.data
@gen_cluster(client=True)
async def dont_test_bad_restrictions_raise_exception(c, s, a, b):
z = c.submit(inc, 2, workers={"bad-address"})
try:
await z
assert False
except ValueError as e:
assert "bad-address" in str(e)
assert z.key in str(e)
@gen_cluster(client=True)
async def test_remove_worker(c, s, a, b):
L = c.map(inc, range(20))
await wait(L)
await b.close()
assert b.address not in s.workers
result = await c.gather(L)
assert result == list(map(inc, range(20)))