/
test_worker.py
3149 lines (2395 loc) · 93.3 KB
/
test_worker.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 importlib
import logging
import os
import sys
import threading
import traceback
from concurrent.futures import ProcessPoolExecutor, ThreadPoolExecutor
from concurrent.futures.process import BrokenProcessPool
from numbers import Number
from operator import add
from time import sleep
from unittest import mock
import psutil
import pytest
from tlz import first, pluck, sliding_window
import dask
from dask import delayed
from dask.system import CPU_COUNT
from dask.utils import tmpfile
import distributed
from distributed import (
Client,
Nanny,
Reschedule,
default_client,
get_client,
get_worker,
wait,
)
from distributed.comm.registry import backends
from distributed.comm.tcp import TCPBackend
from distributed.compatibility import LINUX, WINDOWS
from distributed.core import CommClosedError, Status, rpc
from distributed.diagnostics import nvml
from distributed.diagnostics.plugin import PipInstall
from distributed.metrics import time
from distributed.scheduler import Scheduler
from distributed.utils import TimeoutError
from distributed.utils_test import (
TaskStateMetadataPlugin,
_LockedCommPool,
captured_logger,
dec,
div,
gen_cluster,
gen_test,
inc,
mul,
nodebug,
slowinc,
slowsum,
)
from distributed.worker import Worker, error_message, logger, parse_memory_limit
pytestmark = pytest.mark.ci1
@gen_cluster(nthreads=[])
async def test_worker_nthreads(s):
async with Worker(s.address) as w:
assert w.executor._max_workers == CPU_COUNT
@gen_cluster()
async def test_str(s, a, b):
assert a.address in str(a)
assert a.address in repr(a)
assert str(a.nthreads) in str(a)
assert str(a.nthreads) in repr(a)
assert str(a.executing_count) in repr(a)
@gen_cluster(nthreads=[])
async def test_identity(s):
async with Worker(s.address) as w:
ident = w.identity(None)
assert "Worker" in ident["type"]
assert ident["scheduler"] == s.address
assert isinstance(ident["nthreads"], int)
assert isinstance(ident["memory_limit"], Number)
@gen_cluster(client=True)
async def test_worker_bad_args(c, s, a, b):
class NoReprObj:
"""This object cannot be properly represented as a string."""
def __str__(self):
raise ValueError("I have no str representation.")
def __repr__(self):
raise ValueError("I have no repr representation.")
x = c.submit(NoReprObj, workers=a.address)
await wait(x)
assert not a.executing_count
assert a.data
def bad_func(*args, **kwargs):
1 / 0
class MockLoggingHandler(logging.Handler):
"""Mock logging handler to check for expected logs."""
def __init__(self, *args, **kwargs):
self.reset()
super().__init__(*args, **kwargs)
def emit(self, record):
self.messages[record.levelname.lower()].append(record.getMessage())
def reset(self):
self.messages = {
"debug": [],
"info": [],
"warning": [],
"error": [],
"critical": [],
}
hdlr = MockLoggingHandler()
old_level = logger.level
logger.setLevel(logging.DEBUG)
logger.addHandler(hdlr)
y = c.submit(bad_func, x, k=x, workers=b.address)
await wait(y)
assert not b.executing_count
assert y.status == "error"
# Make sure job died because of bad func and not because of bad
# argument.
with pytest.raises(ZeroDivisionError):
await y
tb = await y._traceback()
assert any("1 / 0" in line for line in pluck(3, traceback.extract_tb(tb)) if line)
assert "Compute Failed" in hdlr.messages["warning"][0]
logger.setLevel(old_level)
# Now we check that both workers are still alive.
xx = c.submit(add, 1, 2, workers=a.address)
yy = c.submit(add, 3, 4, workers=b.address)
results = await c._gather([xx, yy])
assert tuple(results) == (3, 7)
@gen_cluster(client=True)
async def test_upload_file(c, s, a, b):
assert not os.path.exists(os.path.join(a.local_directory, "foobar.py"))
assert not os.path.exists(os.path.join(b.local_directory, "foobar.py"))
assert a.local_directory != b.local_directory
with rpc(a.address) as aa, rpc(b.address) as bb:
await asyncio.gather(
aa.upload_file(filename="foobar.py", data=b"x = 123"),
bb.upload_file(filename="foobar.py", data="x = 123"),
)
assert os.path.exists(os.path.join(a.local_directory, "foobar.py"))
assert os.path.exists(os.path.join(b.local_directory, "foobar.py"))
def g():
import foobar
return foobar.x
future = c.submit(g, workers=a.address)
result = await future
assert result == 123
await c.close()
await s.close(close_workers=True)
assert not os.path.exists(os.path.join(a.local_directory, "foobar.py"))
@pytest.mark.skip(reason="don't yet support uploading pyc files")
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
async def test_upload_file_pyc(c, s, w):
with tmpfile() as dirname:
os.mkdir(dirname)
with open(os.path.join(dirname, "foo.py"), mode="w") as f:
f.write("def f():\n return 123")
sys.path.append(dirname)
try:
import foo
assert foo.f() == 123
pyc = importlib.util.cache_from_source(os.path.join(dirname, "foo.py"))
assert os.path.exists(pyc)
await c.upload_file(pyc)
def g():
import foo
return foo.x
future = c.submit(g)
result = await future
assert result == 123
finally:
sys.path.remove(dirname)
@gen_cluster(client=True)
async def test_upload_egg(c, s, a, b):
eggname = "testegg-1.0.0-py3.4.egg"
local_file = __file__.replace("test_worker.py", eggname)
assert not os.path.exists(os.path.join(a.local_directory, eggname))
assert not os.path.exists(os.path.join(b.local_directory, eggname))
assert a.local_directory != b.local_directory
await c.upload_file(filename=local_file)
assert os.path.exists(os.path.join(a.local_directory, eggname))
assert os.path.exists(os.path.join(b.local_directory, eggname))
def g(x):
import testegg
return testegg.inc(x)
future = c.submit(g, 10, workers=a.address)
result = await future
assert result == 10 + 1
await c.close()
await s.close()
await a.close()
await b.close()
assert not os.path.exists(os.path.join(a.local_directory, eggname))
@gen_cluster(client=True)
async def test_upload_pyz(c, s, a, b):
pyzname = "mytest.pyz"
local_file = __file__.replace("test_worker.py", pyzname)
assert not os.path.exists(os.path.join(a.local_directory, pyzname))
assert not os.path.exists(os.path.join(b.local_directory, pyzname))
assert a.local_directory != b.local_directory
await c.upload_file(filename=local_file)
assert os.path.exists(os.path.join(a.local_directory, pyzname))
assert os.path.exists(os.path.join(b.local_directory, pyzname))
def g(x):
from mytest import mytest
return mytest.inc(x)
future = c.submit(g, 10, workers=a.address)
result = await future
assert result == 10 + 1
await c.close()
await s.close()
await a.close()
await b.close()
assert not os.path.exists(os.path.join(a.local_directory, pyzname))
@pytest.mark.xfail(reason="Still lose time to network I/O")
@gen_cluster(client=True)
async def test_upload_large_file(c, s, a, b):
pytest.importorskip("crick")
await asyncio.sleep(0.05)
with rpc(a.address) as aa:
await aa.upload_file(filename="myfile.dat", data=b"0" * 100000000)
await asyncio.sleep(0.05)
assert a.digests["tick-duration"].components[0].max() < 0.050
@gen_cluster()
async def test_broadcast(s, a, b):
with rpc(s.address) as cc:
results = await cc.broadcast(msg={"op": "ping"})
assert results == {a.address: b"pong", b.address: b"pong"}
@gen_cluster(nthreads=[])
async def test_worker_with_port_zero(s):
async with Worker(s.address) as w:
assert isinstance(w.port, int)
assert w.port > 1024
@gen_cluster(nthreads=[])
async def test_worker_port_range(s):
port = "9867:9868"
async with Worker(s.address, port=port) as w1:
assert w1.port == 9867 # Selects first port in range
async with Worker(s.address, port=port) as w2:
assert w2.port == 9868 # Selects next port in range
with pytest.raises(
ValueError, match="Could not start Worker"
): # No more ports left
async with Worker(s.address, port=port):
pass
@pytest.mark.slow
@gen_test(timeout=60)
async def test_worker_waits_for_scheduler():
w = Worker("127.0.0.1:8724")
try:
await asyncio.wait_for(w, 3)
except TimeoutError:
pass
else:
assert False
assert w.status not in (Status.closed, Status.running, Status.paused)
await w.close(timeout=0.1)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
async def test_worker_task_data(c, s, w):
x = delayed(2)
xx = c.persist(x)
await wait(xx)
assert w.data[x.key] == 2
def test_error_message():
class MyException(Exception):
def __init__(self, a, b):
self.args = (a + b,)
def __str__(self):
return "MyException(%s)" % self.args
msg = error_message(MyException("Hello", "World!"))
assert "Hello" in str(msg["exception"])
max_error_len = 100
with dask.config.set({"distributed.admin.max-error-length": max_error_len}):
msg = error_message(RuntimeError("-" * max_error_len))
assert len(msg["exception_text"]) <= max_error_len + 30
assert len(msg["exception_text"]) < max_error_len * 2
msg = error_message(RuntimeError("-" * max_error_len * 20))
max_error_len = 1000000
with dask.config.set({"distributed.admin.max-error-length": max_error_len}):
msg = error_message(RuntimeError("-" * max_error_len * 2))
assert len(msg["exception_text"]) > 10100 # default + 100
@gen_cluster(client=True)
async def test_chained_error_message(c, s, a, b):
def chained_exception_fn():
class MyException(Exception):
def __init__(self, msg):
self.msg = msg
def __str__(self):
return "MyException(%s)" % self.msg
exception = MyException("Foo")
inner_exception = MyException("Bar")
try:
raise inner_exception
except Exception as e:
raise exception from e
f = c.submit(chained_exception_fn)
try:
await f
except Exception as e:
assert e.__cause__ is not None
assert "Bar" in str(e.__cause__)
@gen_cluster(client=True)
async def test_gather(c, s, a, b):
x, y = await c.scatter(["x", "y"], workers=[b.address])
with rpc(a.address) as aa:
resp = await aa.gather(who_has={x.key: [b.address], y.key: [b.address]})
assert resp == {"status": "OK"}
assert a.data[x.key] == b.data[x.key] == "x"
assert a.data[y.key] == b.data[y.key] == "y"
@gen_cluster(client=True)
async def test_gather_missing_keys(c, s, a, b):
"""A key is missing. Other keys are gathered successfully."""
x = await c.scatter("x", workers=[b.address])
with rpc(a.address) as aa:
resp = await aa.gather(who_has={x.key: [b.address], "y": [b.address]})
assert resp == {"status": "partial-fail", "keys": {"y": (b.address,)}}
assert a.data[x.key] == b.data[x.key] == "x"
@gen_cluster(client=True, worker_kwargs={"timeout": "100ms"})
async def test_gather_missing_workers(c, s, a, b):
"""A worker owning the only copy of a key is missing.
Keys from other workers are gathered successfully.
"""
assert b.address.startswith("tcp://127.0.0.1:")
bad_addr = "tcp://127.0.0.1:12345"
x = await c.scatter("x", workers=[b.address])
with rpc(a.address) as aa:
resp = await aa.gather(who_has={x.key: [b.address], "y": [bad_addr]})
assert resp == {"status": "partial-fail", "keys": {"y": (bad_addr,)}}
assert a.data[x.key] == b.data[x.key] == "x"
@pytest.mark.parametrize("missing_first", [False, True])
@gen_cluster(client=True, worker_kwargs={"timeout": "100ms"})
async def test_gather_missing_workers_replicated(c, s, a, b, missing_first):
"""A worker owning a redundant copy of a key is missing.
The key is successfully gathered from other workers.
"""
assert b.address.startswith("tcp://127.0.0.1:")
x = await c.scatter("x", workers=[b.address])
bad_addr = "tcp://127.0.0.1:12345"
# Order matters! Test both
addrs = [bad_addr, b.address] if missing_first else [b.address, bad_addr]
with rpc(a.address) as aa:
resp = await aa.gather(who_has={x.key: addrs})
assert resp == {"status": "OK"}
assert a.data[x.key] == b.data[x.key] == "x"
@gen_cluster(nthreads=[])
async def test_io_loop(s):
async with Worker(s.address, loop=s.loop) as w:
assert w.io_loop is s.loop
@gen_cluster(client=True, nthreads=[])
async def test_spill_to_disk(c, s):
np = pytest.importorskip("numpy")
w = await Worker(
s.address,
loop=s.loop,
memory_limit=1200 / 0.6,
memory_pause_fraction=None,
memory_spill_fraction=None,
)
x = c.submit(np.random.randint, 0, 255, size=500, dtype="u1", key="x")
await wait(x)
y = c.submit(np.random.randint, 0, 255, size=500, dtype="u1", key="y")
await wait(y)
assert set(w.data) == {x.key, y.key}
assert set(w.data.memory) == {x.key, y.key}
z = c.submit(np.random.randint, 0, 255, size=500, dtype="u1", key="z")
await wait(z)
assert set(w.data) == {x.key, y.key, z.key}
assert set(w.data.memory) == {y.key, z.key}
assert set(w.data.disk) == {x.key}
await x
assert set(w.data.memory) == {x.key, z.key}
assert set(w.data.disk) == {y.key}
await w.close()
@gen_cluster(client=True)
async def test_access_key(c, s, a, b):
def f(i):
from distributed.worker import thread_state
return thread_state.key
futures = [c.submit(f, i, key="x-%d" % i) for i in range(20)]
results = await c._gather(futures)
assert list(results) == ["x-%d" % i for i in range(20)]
@gen_cluster(client=True)
async def test_run_dask_worker(c, s, a, b):
def f(dask_worker=None):
return dask_worker.id
response = await c._run(f)
assert response == {a.address: a.id, b.address: b.id}
@gen_cluster(client=True)
async def test_run_coroutine_dask_worker(c, s, a, b):
async def f(dask_worker=None):
await asyncio.sleep(0.001)
return dask_worker.id
response = await c.run(f)
assert response == {a.address: a.id, b.address: b.id}
@gen_cluster(client=True, nthreads=[])
async def test_Executor(c, s):
with ThreadPoolExecutor(2) as e:
async with Worker(s.address, executor=e) as w:
assert w.executor is e
future = c.submit(inc, 1)
result = await future
assert result == 2
assert e._threads # had to do some work
@pytest.mark.skip(
reason="Other tests leak memory, so process-level checks trigger immediately"
)
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)],
timeout=30,
worker_kwargs={"memory_limit": 10e6},
)
async def test_spill_by_default(c, s, w):
da = pytest.importorskip("dask.array")
x = da.ones(int(10e6 * 0.7), chunks=1e6, dtype="u1")
y = c.persist(x)
await wait(y)
assert len(w.data.disk) # something is on disk
@gen_cluster(nthreads=[("127.0.0.1", 1)], worker_kwargs={"reconnect": False})
async def test_close_on_disconnect(s, w):
await s.close()
start = time()
while w.status != Status.closed:
await asyncio.sleep(0.01)
assert time() < start + 5
@gen_cluster(nthreads=[])
async def test_memory_limit_auto(s):
async with Worker(s.address, nthreads=1) as a, Worker(
s.address, nthreads=2
) as b, Worker(s.address, nthreads=100) as c, Worker(s.address, nthreads=200) as d:
assert isinstance(a.memory_limit, Number)
assert isinstance(b.memory_limit, Number)
if CPU_COUNT > 1:
assert a.memory_limit < b.memory_limit
assert c.memory_limit == d.memory_limit
@gen_cluster(client=True)
async def test_inter_worker_communication(c, s, a, b):
[x, y] = await c._scatter([1, 2], workers=a.address)
future = c.submit(add, x, y, workers=b.address)
result = await future
assert result == 3
@gen_cluster(client=True)
async def test_clean(c, s, a, b):
x = c.submit(inc, 1, workers=a.address)
y = c.submit(inc, x, workers=b.address)
await y
collections = [
a.tasks,
a.data,
a.threads,
]
for c in collections:
assert c
x.release()
y.release()
while x.key in a.tasks:
await asyncio.sleep(0.01)
for c in collections:
assert not c
@gen_cluster(client=True)
async def test_message_breakup(c, s, a, b):
n = 100000
a.target_message_size = 10 * n
b.target_message_size = 10 * n
xs = [c.submit(mul, b"%d" % i, n, workers=a.address) for i in range(30)]
y = c.submit(lambda *args: None, xs, workers=b.address)
await y
assert 2 <= len(b.incoming_transfer_log) <= 20
assert 2 <= len(a.outgoing_transfer_log) <= 20
assert all(msg["who"] == b.address for msg in a.outgoing_transfer_log)
assert all(msg["who"] == a.address for msg in a.incoming_transfer_log)
@gen_cluster(client=True)
async def test_types(c, s, a, b):
assert all(ts.type is None for ts in a.tasks.values())
assert all(ts.type is None for ts in b.tasks.values())
x = c.submit(inc, 1, workers=a.address)
await wait(x)
assert a.tasks[x.key].type == int
y = c.submit(inc, x, workers=b.address)
await wait(y)
assert b.tasks[x.key].type == int
assert b.tasks[y.key].type == int
await c._cancel(y)
start = time()
while y.key in b.data:
await asyncio.sleep(0.01)
assert time() < start + 5
assert y.key not in b.tasks
@gen_cluster()
async def test_system_monitor(s, a, b):
assert b.monitor
b.monitor.update()
@gen_cluster(
client=True, nthreads=[("127.0.0.1", 2, {"resources": {"A": 1}}), ("127.0.0.1", 1)]
)
async def test_restrictions(c, s, a, b):
# Worker has resource available
assert a.available_resources == {"A": 1}
# Resource restrictions
x = c.submit(inc, 1, resources={"A": 1})
await x
ts = a.tasks[x.key]
assert ts.resource_restrictions == {"A": 1}
await c._cancel(x)
while ts.state != "memory":
# Resource should be unavailable while task isn't finished
assert a.available_resources == {"A": 0}
await asyncio.sleep(0.01)
# Resource restored after task is in memory
assert a.available_resources["A"] == 1
@gen_cluster(client=True)
async def test_clean_nbytes(c, s, a, b):
L = [delayed(inc)(i) for i in range(10)]
for i in range(5):
L = [delayed(add)(x, y) for x, y in sliding_window(2, L)]
total = delayed(sum)(L)
future = c.compute(total)
await wait(future)
await asyncio.sleep(1)
assert (
len(list(filter(None, [ts.nbytes for ts in a.tasks.values()])))
+ len(list(filter(None, [ts.nbytes for ts in b.tasks.values()])))
== 1
)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 20)
async def test_gather_many_small(c, s, a, *workers):
"""If the dependencies of a given task are very small, do not limit the
number of concurrent outgoing connections
"""
a.total_out_connections = 2
futures = await c._scatter(list(range(100)))
assert all(w.data for w in workers)
def f(*args):
return 10
future = c.submit(f, *futures, workers=a.address)
await wait(future)
types = list(pluck(0, a.log))
req = [i for i, t in enumerate(types) if t == "request-dep"]
recv = [i for i, t in enumerate(types) if t == "receive-dep"]
assert min(recv) > max(req)
assert a.comm_nbytes == 0
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_multiple_transfers(c, s, w1, w2, w3):
x = c.submit(inc, 1, workers=w1.address)
y = c.submit(inc, 2, workers=w2.address)
z = c.submit(add, x, y, workers=w3.address)
await wait(z)
r = w3.tasks[z.key].startstops
transfers = [t for t in r if t["action"] == "transfer"]
assert len(transfers) == 2
@pytest.mark.xfail(reason="very high flakiness")
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_share_communication(c, s, w1, w2, w3):
x = c.submit(mul, b"1", int(w3.target_message_size + 1), workers=w1.address)
y = c.submit(mul, b"2", int(w3.target_message_size + 1), workers=w2.address)
await wait([x, y])
await c._replicate([x, y], workers=[w1.address, w2.address])
z = c.submit(add, x, y, workers=w3.address)
await wait(z)
assert len(w3.incoming_transfer_log) == 2
assert w1.outgoing_transfer_log
assert w2.outgoing_transfer_log
@pytest.mark.xfail(reason="very high flakiness")
@gen_cluster(client=True)
async def test_dont_overlap_communications_to_same_worker(c, s, a, b):
x = c.submit(mul, b"1", int(b.target_message_size + 1), workers=a.address)
y = c.submit(mul, b"2", int(b.target_message_size + 1), workers=a.address)
await wait([x, y])
z = c.submit(add, x, y, workers=b.address)
await wait(z)
assert len(b.incoming_transfer_log) == 2
l1, l2 = b.incoming_transfer_log
assert l1["stop"] < l2["start"]
@gen_cluster(client=True)
async def test_log_exception_on_failed_task(c, s, a, b):
with captured_logger("distributed.worker") as logger:
future = c.submit(div, 1, 0)
await wait(future)
await asyncio.sleep(0.1)
text = logger.getvalue()
assert "ZeroDivisionError" in text
assert "Exception" in text
@gen_cluster(client=True)
async def test_clean_up_dependencies(c, s, a, b):
x = delayed(inc)(1)
y = delayed(inc)(2)
xx = delayed(inc)(x)
yy = delayed(inc)(y)
z = delayed(add)(xx, yy)
zz = c.persist(z)
await wait(zz)
while len(a.data) + len(b.data) > 1:
await asyncio.sleep(0.01)
assert set(a.data) | set(b.data) == {zz.key}
@gen_cluster(client=True)
async def test_hold_onto_dependents(c, s, a, b):
x = c.submit(inc, 1, workers=a.address)
y = c.submit(inc, x, workers=b.address)
await wait(y)
assert x.key in b.data
await c._cancel(y)
while x.key not in b.data:
await asyncio.sleep(0.1)
# Normally takes >2s but it has been observed to take >30s occasionally
@pytest.mark.slow
@gen_test(timeout=120)
async def test_worker_death_timeout():
w = Worker("tcp://127.0.0.1:12345", death_timeout=0.1)
with pytest.raises(TimeoutError) as info:
await w
assert "Worker" in str(info.value)
assert "timed out" in str(info.value) or "failed to start" in str(info.value)
assert w.status == Status.closed
@gen_cluster(client=True)
async def test_stop_doing_unnecessary_work(c, s, a, b):
futures = c.map(slowinc, range(1000), delay=0.01)
await asyncio.sleep(0.1)
del futures
start = time()
while a.executing_count:
await asyncio.sleep(0.01)
assert time() - start < 0.5
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
async def test_priorities(c, s, w):
values = []
for i in range(10):
a = delayed(slowinc)(i, dask_key_name="a-%d" % i, delay=0.01)
a1 = delayed(inc)(a, dask_key_name="a1-%d" % i)
a2 = delayed(inc)(a1, dask_key_name="a2-%d" % i)
b1 = delayed(dec)(a, dask_key_name="b1-%d" % i) # <<-- least favored
values.append(a2)
values.append(b1)
futures = c.compute(values)
await wait(futures)
log = [
t[0]
for t in w.log
if t[1] == "executing" and t[2] == "memory" and not t[0].startswith("finalize")
]
assert any(key.startswith("b1") for key in log[: len(log) // 2])
@gen_cluster(client=True)
async def test_heartbeats(c, s, a, b):
x = s.workers[a.address].last_seen
start = time()
await asyncio.sleep(a.periodic_callbacks["heartbeat"].callback_time / 1000 + 0.1)
while s.workers[a.address].last_seen == x:
await asyncio.sleep(0.01)
assert time() < start + 2
assert a.periodic_callbacks["heartbeat"].callback_time < 1000
@pytest.mark.parametrize("worker", [Worker, Nanny])
def test_worker_dir(worker):
with tmpfile() as fn:
@gen_cluster(client=True, worker_kwargs={"local_directory": fn})
async def test_worker_dir(c, s, a, b):
directories = [w.local_directory for w in s.workers.values()]
assert all(d.startswith(fn) for d in directories)
assert len(set(directories)) == 2 # distinct
test_worker_dir()
@gen_cluster(nthreads=[])
async def test_false_worker_dir(s):
async with Worker(s.address, local_directory="") as w:
local_directory = w.local_directory
cwd = os.getcwd()
assert os.path.dirname(local_directory) == os.path.join(cwd, "dask-worker-space")
@gen_cluster(client=True)
async def test_dataframe_attribute_error(c, s, a, b):
class BadSize:
def __init__(self, data):
self.data = data
def __sizeof__(self):
raise TypeError("Hello")
future = c.submit(BadSize, 123)
result = await future
assert result.data == 123
@gen_cluster(client=True)
async def test_fail_write_to_disk(c, s, a, b):
class Bad:
def __getstate__(self):
raise TypeError()
def __sizeof__(self):
return int(100e9)
future = c.submit(Bad)
await wait(future)
assert future.status == "error"
with pytest.raises(TypeError):
await future
futures = c.map(inc, range(10))
results = await c._gather(futures)
assert results == list(map(inc, range(10)))
@pytest.mark.skip(reason="Our logic here is faulty")
@gen_cluster(
nthreads=[("127.0.0.1", 2)], client=True, worker_kwargs={"memory_limit": 10e9}
)
async def test_fail_write_many_to_disk(c, s, a):
a.validate = False
await asyncio.sleep(0.1)
assert a.status == Status.running
class Bad:
def __init__(self, x):
pass
def __getstate__(self):
raise TypeError()
def __sizeof__(self):
return int(2e9)
futures = c.map(Bad, range(11))
future = c.submit(lambda *args: 123, *futures)
await wait(future)
with pytest.raises(Exception) as info:
await future
# workers still operational
result = await c.submit(inc, 1, workers=a.address)
assert result == 2
@gen_cluster()
async def test_pid(s, a, b):
assert s.workers[a.address].pid == os.getpid()
@gen_cluster(client=True)
async def test_get_client(c, s, a, b):
def f(x):
cc = get_client()
future = cc.submit(inc, x)
return future.result()
assert default_client() is c
future = c.submit(f, 10, workers=a.address)
result = await future
assert result == 11
assert a._client
assert not b._client
assert a._client is c
assert default_client() is c
a_client = a._client
for i in range(10):
await wait(c.submit(f, i))
assert a._client is a_client
def test_get_client_sync(client):
def f(x):
cc = get_client()
future = cc.submit(inc, x)
return future.result()
future = client.submit(f, 10)
assert future.result() == 11
@gen_cluster(client=True)
async def test_get_client_coroutine(c, s, a, b):
async def f():
client = await get_client()
future = client.submit(inc, 10)
result = await future
return result
results = await c.run(f)
assert results == {a.address: 11, b.address: 11}
def test_get_client_coroutine_sync(client, s, a, b):
async def f():
client = await get_client()
future = client.submit(inc, 10)
result = await future
return result
results = client.run(f)
assert results == {a["address"]: 11, b["address"]: 11}