forked from dask/distributed
-
Notifications
You must be signed in to change notification settings - Fork 0
/
test_worker.py
1441 lines (1082 loc) · 38.5 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
from __future__ import print_function, division, absolute_import
from concurrent.futures import ThreadPoolExecutor
from datetime import timedelta
import logging
import multiprocessing
from numbers import Number
from operator import add
import os
import psutil
import shutil
import sys
from time import sleep
import traceback
import dask
from dask import delayed
import pytest
from toolz import pluck, sliding_window, first
import tornado
from tornado import gen
from tornado.ioloop import TimeoutError
from distributed import Nanny, get_client, wait, default_client, get_worker, Reschedule
from distributed.compatibility import WINDOWS, cache_from_source
from distributed.core import rpc
from distributed.client import wait
from distributed.scheduler import Scheduler
from distributed.metrics import time
from distributed.worker import Worker, error_message, logger, parse_memory_limit
from distributed.utils import tmpfile, format_bytes
from distributed.utils_test import (
inc,
mul,
gen_cluster,
div,
dec,
slowinc,
gen_test,
captured_logger,
)
from distributed.utils_test import ( # noqa: F401
client,
loop,
nodebug,
cluster_fixture,
s,
a,
b,
)
def test_worker_nthreads():
w = Worker("127.0.0.1", 8019)
try:
assert w.executor._max_workers == multiprocessing.cpu_count()
finally:
shutil.rmtree(w.local_dir)
@gen_cluster()
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(len(a.executing)) in repr(a)
def test_identity():
w = Worker("127.0.0.1", 8019)
ident = w.identity(None)
assert "Worker" in ident["type"]
assert ident["scheduler"] == "tcp://127.0.0.1:8019"
assert isinstance(ident["nthreads"], int)
assert isinstance(ident["memory_limit"], Number)
@gen_cluster(client=True)
def test_worker_bad_args(c, s, a, b):
class NoReprObj(object):
""" 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)
yield wait(x)
assert not a.executing
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()
logging.Handler.__init__(self, *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)
yield wait(y)
assert not b.executing
assert y.status == "error"
# Make sure job died because of bad func and not because of bad
# argument.
with pytest.raises(ZeroDivisionError):
yield y
if sys.version_info[0] >= 3:
tb = yield 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 = yield c._gather([xx, yy])
assert tuple(results) == (3, 7)
@pytest.mark.slow
@gen_cluster()
def dont_test_delete_data_with_missing_worker(c, a, b):
bad = "127.0.0.1:9001" # this worker doesn't exist
c.who_has["z"].add(bad)
c.who_has["z"].add(a.address)
c.has_what[bad].add("z")
c.has_what[a.address].add("z")
a.data["z"] = 5
cc = rpc(ip=c.ip, port=c.port)
yield cc.delete_data(keys=["z"]) # TODO: this hangs for a while
assert "z" not in a.data
assert not c.who_has["z"]
assert not c.has_what[bad]
assert not c.has_what[a.address]
cc.close_rpc()
@gen_cluster(client=True)
def test_upload_file(c, s, a, b):
assert not os.path.exists(os.path.join(a.local_dir, "foobar.py"))
assert not os.path.exists(os.path.join(b.local_dir, "foobar.py"))
assert a.local_dir != b.local_dir
with rpc(a.address) as aa, rpc(b.address) as bb:
yield [
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_dir, "foobar.py"))
assert os.path.exists(os.path.join(b.local_dir, "foobar.py"))
def g():
import foobar
return foobar.x
future = c.submit(g, workers=a.address)
result = yield future
assert result == 123
yield c.close()
yield s.close(close_workers=True)
assert not os.path.exists(os.path.join(a.local_dir, "foobar.py"))
@pytest.mark.skip(reason="don't yet support uploading pyc files")
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
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 = cache_from_source(os.path.join(dirname, "foo.py"))
assert os.path.exists(pyc)
yield c.upload_file(pyc)
def g():
import foo
return foo.x
future = c.submit(g)
result = yield future
assert result == 123
finally:
sys.path.remove(dirname)
@gen_cluster(client=True)
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_dir, eggname))
assert not os.path.exists(os.path.join(b.local_dir, eggname))
assert a.local_dir != b.local_dir
yield c.upload_file(filename=local_file)
assert os.path.exists(os.path.join(a.local_dir, eggname))
assert os.path.exists(os.path.join(b.local_dir, eggname))
def g(x):
import testegg
return testegg.inc(x)
future = c.submit(g, 10, workers=a.address)
result = yield future
assert result == 10 + 1
yield c.close()
yield s.close()
yield a.close()
yield b.close()
assert not os.path.exists(os.path.join(a.local_dir, eggname))
@gen_cluster(client=True)
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_dir, pyzname))
assert not os.path.exists(os.path.join(b.local_dir, pyzname))
assert a.local_dir != b.local_dir
yield c.upload_file(filename=local_file)
assert os.path.exists(os.path.join(a.local_dir, pyzname))
assert os.path.exists(os.path.join(b.local_dir, pyzname))
def g(x):
from mytest import mytest
return mytest.inc(x)
future = c.submit(g, 10, workers=a.address)
result = yield future
assert result == 10 + 1
yield c.close()
yield s.close()
yield a.close()
yield b.close()
assert not os.path.exists(os.path.join(a.local_dir, pyzname))
@pytest.mark.xfail(reason="Still lose time to network I/O")
@gen_cluster(client=True)
def test_upload_large_file(c, s, a, b):
pytest.importorskip("crick")
yield gen.sleep(0.05)
with rpc(a.address) as aa:
yield aa.upload_file(filename="myfile.dat", data=b"0" * 100000000)
yield gen.sleep(0.05)
assert a.digests["tick-duration"].components[0].max() < 0.050
@gen_cluster()
def test_broadcast(s, a, b):
with rpc(s.address) as cc:
results = yield cc.broadcast(msg={"op": "ping"})
assert results == {a.address: b"pong", b.address: b"pong"}
@gen_test()
def test_worker_with_port_zero():
s = yield Scheduler(port=8007)
w = yield Worker(s.address)
assert isinstance(w.port, int)
assert w.port > 1024
yield w.close()
@pytest.mark.slow
def test_worker_waits_for_scheduler(loop):
@gen.coroutine
def f():
w = Worker("127.0.0.1", 8007)
try:
yield gen.with_timeout(timedelta(seconds=3), w)
except TimeoutError:
pass
else:
assert False
assert w.status not in ("closed", "running")
yield w.close(timeout=0.1)
loop.run_sync(f)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
def test_worker_task_data(c, s, w):
x = delayed(2)
xx = c.persist(x)
yield 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"])
@gen_cluster()
def test_gather(s, a, b):
b.data["x"] = 1
b.data["y"] = 2
with rpc(a.address) as aa:
resp = yield aa.gather(who_has={"x": [b.address], "y": [b.address]})
assert resp["status"] == "OK"
assert a.data["x"] == b.data["x"]
assert a.data["y"] == b.data["y"]
@pytest.mark.asyncio
async def test_io_loop():
s = await Scheduler(port=0)
w = await Worker(s.address, loop=s.loop)
assert w.io_loop is s.loop
await s.close()
await w.close()
@gen_cluster(client=True, nthreads=[])
def test_spill_to_disk(c, s):
np = pytest.importorskip("numpy")
w = yield 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")
yield wait(x)
y = c.submit(np.random.randint, 0, 255, size=500, dtype="u1", key="y")
yield wait(y)
assert set(w.data) == {x.key, y.key}
assert set(w.data.memory) == {x.key, y.key}
assert set(w.data.fast) == set(w.data.memory)
z = c.submit(np.random.randint, 0, 255, size=500, dtype="u1", key="z")
yield 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} or set(w.data.slow) == {x.key, y.key}
assert set(w.data.fast) == set(w.data.memory)
assert set(w.data.slow) == set(w.data.disk)
yield x
assert set(w.data.memory) == {x.key, z.key}
assert set(w.data.disk) == {y.key} or set(w.data.slow) == {x.key, y.key}
assert set(w.data.fast) == set(w.data.memory)
assert set(w.data.slow) == set(w.data.disk)
yield w.close()
@gen_cluster(client=True)
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 = yield c._gather(futures)
assert list(results) == ["x-%d" % i for i in range(20)]
@gen_cluster(client=True)
def test_run_dask_worker(c, s, a, b):
def f(dask_worker=None):
return dask_worker.id
response = yield c._run(f)
assert response == {a.address: a.id, b.address: b.id}
@gen_cluster(client=True)
def test_run_coroutine_dask_worker(c, s, a, b):
if sys.version_info < (3,) and tornado.version_info < (4, 5):
pytest.skip("test needs Tornado 4.5+ on Python 2.7")
@gen.coroutine
def f(dask_worker=None):
yield gen.sleep(0.001)
raise gen.Return(dask_worker.id)
response = yield c.run(f)
assert response == {a.address: a.id, b.address: b.id}
@gen_cluster(client=True, nthreads=[])
def test_Executor(c, s):
with ThreadPoolExecutor(2) as e:
w = Worker(s.address, executor=e)
assert w.executor is e
w = yield w
future = c.submit(inc, 1)
result = yield future
assert result == 2
assert e._threads # had to do some work
yield w.close()
@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},
)
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)
yield wait(y)
assert len(w.data.disk) # something is on disk
del x, y
@gen_cluster(nthreads=[("127.0.0.1", 1)], worker_kwargs={"reconnect": False})
def test_close_on_disconnect(s, w):
yield s.close()
start = time()
while w.status != "closed":
yield gen.sleep(0.01)
assert time() < start + 5
def test_memory_limit_auto():
a = Worker("127.0.0.1", 8099, nthreads=1)
b = Worker("127.0.0.1", 8099, nthreads=2)
c = Worker("127.0.0.1", 8099, nthreads=100)
d = Worker("127.0.0.1", 8099, nthreads=200)
assert isinstance(a.memory_limit, Number)
assert isinstance(b.memory_limit, Number)
assert a.memory_limit < b.memory_limit
assert c.memory_limit == d.memory_limit
@gen_cluster(client=True)
def test_inter_worker_communication(c, s, a, b):
[x, y] = yield c._scatter([1, 2], workers=a.address)
future = c.submit(add, x, y, workers=b.address)
result = yield future
assert result == 3
@gen_cluster(client=True)
def test_clean(c, s, a, b):
x = c.submit(inc, 1, workers=a.address)
y = c.submit(inc, x, workers=b.address)
yield y
collections = [
a.tasks,
a.task_state,
a.startstops,
a.data,
a.nbytes,
a.durations,
a.priorities,
a.types,
a.threads,
]
for c in collections:
assert c
x.release()
y.release()
while x.key in a.task_state:
yield gen.sleep(0.01)
for c in collections:
assert not c
@pytest.mark.skipif(sys.version_info[:2] == (3, 4), reason="mul bytes fails")
@gen_cluster(client=True)
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)
yield 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)
def test_types(c, s, a, b):
assert not a.types
assert not b.types
x = c.submit(inc, 1, workers=a.address)
yield wait(x)
assert a.types[x.key] == int
y = c.submit(inc, x, workers=b.address)
yield wait(y)
assert b.types == {x.key: int, y.key: int}
yield c._cancel(y)
start = time()
while y.key in b.data:
yield gen.sleep(0.01)
assert time() < start + 5
assert y.key not in b.types
@gen_cluster()
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)]
)
def test_restrictions(c, s, a, b):
# Resource restrictions
x = c.submit(inc, 1, resources={"A": 1})
yield x
assert a.resource_restrictions == {x.key: {"A": 1}}
yield c._cancel(x)
while x.key in a.task_state:
yield gen.sleep(0.01)
assert a.resource_restrictions == {}
@pytest.mark.xfail
@gen_cluster(client=True)
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)
yield wait(future)
yield gen.sleep(1)
assert len(a.nbytes) + len(b.nbytes) == 1
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 20)
def test_gather_many_small(c, s, a, *workers):
a.total_out_connections = 2
futures = yield 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)
yield 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)
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)
yield wait(z)
r = w3.startstops[z.key]
transfers = [t for t in r if t[0] == "transfer"]
assert len(transfers) == 2
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
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)
yield wait([x, y])
yield c._replicate([x, y], workers=[w1.address, w2.address])
z = c.submit(add, x, y, workers=w3.address)
yield wait(z)
assert len(w3.incoming_transfer_log) == 2
assert w1.outgoing_transfer_log
assert w2.outgoing_transfer_log
@gen_cluster(client=True)
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)
yield wait([x, y])
z = c.submit(add, x, y, workers=b.address)
yield wait(z)
assert len(b.incoming_transfer_log) == 2
l1, l2 = b.incoming_transfer_log
assert l1["stop"] < l2["start"]
@pytest.mark.avoid_travis
@gen_cluster(client=True)
def test_log_exception_on_failed_task(c, s, a, b):
with tmpfile() as fn:
fh = logging.FileHandler(fn)
try:
from distributed.worker import logger
logger.addHandler(fh)
future = c.submit(div, 1, 0)
yield wait(future)
yield gen.sleep(0.1)
fh.flush()
with open(fn) as f:
text = f.read()
assert "ZeroDivisionError" in text
assert "Exception" in text
finally:
logger.removeHandler(fh)
@gen_cluster(client=True)
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)
yield wait(zz)
start = time()
while len(a.data) + len(b.data) > 1:
yield gen.sleep(0.01)
assert time() < start + 2
assert set(a.data) | set(b.data) == {zz.key}
@gen_cluster(client=True)
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)
yield wait(y)
assert x.key in b.data
yield c._cancel(y)
yield gen.sleep(0.1)
assert x.key in b.data
@pytest.mark.slow
@gen_cluster(client=False, nthreads=[])
def test_worker_death_timeout(s):
with dask.config.set({"distributed.comm.timeouts.connect": "1s"}):
yield s.close()
w = Worker(s.address, death_timeout=1)
with pytest.raises(gen.TimeoutError):
yield w
assert w.status == "closed"
@gen_cluster(client=True)
def test_stop_doing_unnecessary_work(c, s, a, b):
futures = c.map(slowinc, range(1000), delay=0.01)
yield gen.sleep(0.1)
del futures
start = time()
while a.executing:
yield gen.sleep(0.01)
assert time() - start < 0.5
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)])
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)
yield 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)
def test_heartbeats(c, s, a, b):
x = s.workers[a.address].last_seen
start = time()
yield gen.sleep(a.periodic_callbacks["heartbeat"].callback_time / 1000 + 0.1)
while s.workers[a.address].last_seen == x:
yield gen.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_dir": fn})
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(client=True)
def test_dataframe_attribute_error(c, s, a, b):
class BadSize(object):
def __init__(self, data):
self.data = data
def __sizeof__(self):
raise TypeError("Hello")
future = c.submit(BadSize, 123)
result = yield future
assert result.data == 123
@gen_cluster(client=True)
def test_fail_write_to_disk(c, s, a, b):
class Bad(object):
def __getstate__(self):
raise TypeError()
def __sizeof__(self):
return int(100e9)
future = c.submit(Bad)
yield wait(future)
assert future.status == "error"
with pytest.raises(TypeError):
yield future
futures = c.map(inc, range(10))
results = yield 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}
)
def test_fail_write_many_to_disk(c, s, a):
a.validate = False
yield gen.sleep(0.1)
assert not a.paused
class Bad(object):
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)
yield wait(future)
with pytest.raises(Exception) as info:
yield future
# workers still operational
result = yield c.submit(inc, 1, workers=a.address)
assert result == 2
@gen_cluster()
def test_pid(s, a, b):
assert s.workers[a.address].pid == os.getpid()
@gen_cluster(client=True)
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 = yield 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):
yield 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)
def test_get_client_coroutine(c, s, a, b):
@gen.coroutine
def f():
client = yield get_client()
future = client.submit(inc, 10)
result = yield future
raise gen.Return(result)
results = yield c.run(f)
assert results == {a.address: 11, b.address: 11}
def test_get_client_coroutine_sync(client, s, a, b):
@gen.coroutine
def f():
client = yield get_client()
future = client.submit(inc, 10)
result = yield future
raise gen.Return(result)
results = client.run(f)
assert results == {a["address"]: 11, b["address"]: 11}
@gen_cluster()
def test_global_workers(s, a, b):
n = len(Worker._instances)
w = first(Worker._instances)
assert w is a or w is b
@pytest.mark.skipif(WINDOWS, reason="file descriptors")
@gen_cluster(nthreads=[])
def test_worker_fds(s):
psutil = pytest.importorskip("psutil")
yield gen.sleep(0.05)
start = psutil.Process().num_fds()
worker = yield Worker(s.address, loop=s.loop)
yield gen.sleep(0.1)
middle = psutil.Process().num_fds()
start = time()
while middle > start:
yield gen.sleep(0.01)
assert time() < start + 1
yield worker.close()
start = time()
while psutil.Process().num_fds() > start:
yield gen.sleep(0.01)
assert time() < start + 0.5
@gen_cluster(nthreads=[])
def test_service_hosts_match_worker(s):
pytest.importorskip("bokeh")
from distributed.dashboard import BokehWorker
services = {("dashboard", ":0"): BokehWorker}
w = Worker(s.address, services={("dashboard", ":0"): BokehWorker})
yield w._start("tcp://0.0.0.0")
sock = first(w.services["dashboard"].server._http._sockets.values())
assert sock.getsockname()[0] in ("::", "0.0.0.0")
yield w.close()
w = Worker(s.address, services={("dashboard", ":0"): BokehWorker})
yield w._start("tcp://127.0.0.1")
sock = first(w.services["dashboard"].server._http._sockets.values())
assert sock.getsockname()[0] in ("::", "0.0.0.0")
yield w.close()
w = Worker(s.address, services={("dashboard", 0): BokehWorker})
yield w._start("tcp://127.0.0.1")
sock = first(w.services["dashboard"].server._http._sockets.values())
assert sock.getsockname()[0] == "127.0.0.1"
yield w.close()
@gen_cluster(nthreads=[])
def test_start_services(s):