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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257 | import asyncio
import os
import sys
import threading
import types
import warnings
from functools import partial
import dask
import distributed
import pkg_resources
import pytest
from distributed.comm import (
CommClosedError,
connect,
get_address_host,
get_local_address_for,
inproc,
listen,
parse_address,
parse_host_port,
resolve_address,
tcp,
unparse_host_port,
)
from distributed.comm.registry import backends, get_backend
from distributed.comm.tcp import TCP, TCPBackend, TCPConnector
from distributed.metrics import time
from distributed.protocol import Serialized, deserialize, serialize, to_serialize
from distributed.utils import get_ip, get_ipv6
from distributed.utils_test import loop # noqa: F401
from distributed.utils_test import (
get_cert,
get_client_ssl_context,
get_server_ssl_context,
has_ipv6,
requires_ipv6,
)
from tornado import ioloop
from tornado.concurrent import Future
EXTERNAL_IP4 = get_ip()
if has_ipv6():
with warnings.catch_warnings(record=True):
warnings.simplefilter("always")
EXTERNAL_IP6 = get_ipv6()
ca_file = get_cert("tls-ca-cert.pem")
# The Subject field of our test certs
cert_subject = (
(("countryName", "XY"),),
(("localityName", "Dask-distributed"),),
(("organizationName", "Dask"),),
(("commonName", "localhost"),),
)
def check_tls_extra(info):
assert isinstance(info, dict)
assert info["peercert"]["subject"] == cert_subject
assert "cipher" in info
cipher_name, proto_name, secret_bits = info["cipher"]
# Most likely
assert "AES" in cipher_name
assert "TLS" in proto_name
assert secret_bits >= 128
tls_kwargs = dict(
listen_args={"ssl_context": get_server_ssl_context()},
connect_args={"ssl_context": get_client_ssl_context()},
)
@pytest.mark.asyncio
async def get_comm_pair(listen_addr, listen_args={}, connect_args={}, **kwargs):
q = asyncio.Queue()
async def handle_comm(comm):
await q.put(comm)
listener = await listen(listen_addr, handle_comm, **listen_args, **kwargs)
comm = await connect(listener.contact_address, **connect_args, **kwargs)
serv_comm = await q.get()
return (comm, serv_comm)
def get_tcp_comm_pair(**kwargs):
return get_comm_pair("tcp://", **kwargs)
def get_tls_comm_pair(**kwargs):
kwargs.update(tls_kwargs)
return get_comm_pair("tls://", **kwargs)
def get_inproc_comm_pair(**kwargs):
return get_comm_pair("inproc://", **kwargs)
async def debug_loop():
"""
Debug helper
"""
while True:
loop = ioloop.IOLoop.current()
print(".", loop, loop._handlers)
await asyncio.sleep(0.50)
#
# Test utility functions
#
def test_parse_host_port():
f = parse_host_port
assert f("localhost:123") == ("localhost", 123)
assert f("127.0.0.1:456") == ("127.0.0.1", 456)
assert f("localhost:123", 80) == ("localhost", 123)
assert f("localhost", 80) == ("localhost", 80)
with pytest.raises(ValueError):
f("localhost")
assert f("[::1]:123") == ("::1", 123)
assert f("[fe80::1]:123", 80) == ("fe80::1", 123)
assert f("[::1]", 80) == ("::1", 80)
with pytest.raises(ValueError):
f("[::1]")
with pytest.raises(ValueError):
f("::1:123")
with pytest.raises(ValueError):
f("::1")
def test_unparse_host_port():
f = unparse_host_port
assert f("localhost", 123) == "localhost:123"
assert f("127.0.0.1", 123) == "127.0.0.1:123"
assert f("::1", 123) == "[::1]:123"
assert f("[::1]", 123) == "[::1]:123"
assert f("127.0.0.1") == "127.0.0.1"
assert f("127.0.0.1", None) == "127.0.0.1"
assert f("127.0.0.1", "*") == "127.0.0.1:*"
assert f("::1") == "[::1]"
assert f("[::1]") == "[::1]"
assert f("::1", "*") == "[::1]:*"
def test_get_address_host():
f = get_address_host
assert f("tcp://127.0.0.1:123") == "127.0.0.1"
assert f("inproc://%s/%d/123" % (get_ip(), os.getpid())) == get_ip()
def test_resolve_address():
f = resolve_address
assert f("tcp://127.0.0.1:123") == "tcp://127.0.0.1:123"
assert f("127.0.0.2:789") == "tcp://127.0.0.2:789"
assert f("tcp://0.0.0.0:456") == "tcp://0.0.0.0:456"
assert f("tcp://0.0.0.0:456") == "tcp://0.0.0.0:456"
if has_ipv6():
assert f("tcp://[::1]:123") == "tcp://[::1]:123"
assert f("tls://[::1]:123") == "tls://[::1]:123"
# OS X returns '::0.0.0.2' as canonical representation
assert f("[::2]:789") in ("tcp://[::2]:789", "tcp://[::0.0.0.2]:789")
assert f("tcp://[::]:123") == "tcp://[::]:123"
assert f("localhost:123") == "tcp://127.0.0.1:123"
assert f("tcp://localhost:456") == "tcp://127.0.0.1:456"
assert f("tls://localhost:456") == "tls://127.0.0.1:456"
def test_get_local_address_for():
f = get_local_address_for
assert f("tcp://127.0.0.1:80") == "tcp://127.0.0.1"
assert f("tcp://8.8.8.8:4444") == "tcp://" + get_ip()
if has_ipv6():
assert f("tcp://[::1]:123") == "tcp://[::1]"
inproc_arg = "inproc://%s/%d/444" % (get_ip(), os.getpid())
inproc_res = f(inproc_arg)
assert inproc_res.startswith("inproc://")
assert inproc_res != inproc_arg
#
# Test concrete transport APIs
#
@pytest.mark.asyncio
async def test_tcp_listener_does_not_call_handler_on_handshake_error():
handle_comm_called = False
async def handle_comm(comm):
nonlocal handle_comm_called
handle_comm_called = True
with dask.config.set({"distributed.comm.timeouts.connect": 0.01}):
listener = await tcp.TCPListener("127.0.0.1", handle_comm)
host, port = listener.get_host_port()
# connect without handshake:
reader, writer = await asyncio.open_connection(host=host, port=port)
# wait a bit to let the listener side hit the timeout on the handshake:
await asyncio.sleep(0.02)
assert not handle_comm_called
writer.close()
if hasattr(writer, "wait_closed"): # always true for python >= 3.7, but not for 3.6
await writer.wait_closed()
@pytest.mark.asyncio
async def test_tcp_specific():
"""
Test concrete TCP API.
"""
async def handle_comm(comm):
assert comm.peer_address.startswith("tcp://" + host)
assert comm.extra_info == {}
msg = await comm.read()
msg["op"] = "pong"
await comm.write(msg)
await comm.close()
listener = await tcp.TCPListener("127.0.0.1", handle_comm)
host, port = listener.get_host_port()
assert host in ("localhost", "127.0.0.1", "::1")
assert port > 0
l = []
async def client_communicate(key, delay=0):
addr = "%s:%d" % (host, port)
comm = await connect(listener.contact_address)
assert comm.peer_address == "tcp://" + addr
assert comm.extra_info == {}
await comm.write({"op": "ping", "data": key})
if delay:
await asyncio.sleep(delay)
msg = await comm.read()
assert msg == {"op": "pong", "data": key}
l.append(key)
await comm.close()
await client_communicate(key=1234)
# Many clients at once
N = 100
futures = [client_communicate(key=i, delay=0.05) for i in range(N)]
await asyncio.gather(*futures)
assert set(l) == {1234} | set(range(N))
@pytest.mark.asyncio
async def test_tls_specific():
"""
Test concrete TLS API.
"""
async def handle_comm(comm):
assert comm.peer_address.startswith("tls://" + host)
check_tls_extra(comm.extra_info)
msg = await comm.read()
msg["op"] = "pong"
await comm.write(msg)
await comm.close()
server_ctx = get_server_ssl_context()
client_ctx = get_client_ssl_context()
listener = await tcp.TLSListener("127.0.0.1", handle_comm, ssl_context=server_ctx)
host, port = listener.get_host_port()
assert host in ("localhost", "127.0.0.1", "::1")
assert port > 0
l = []
async def client_communicate(key, delay=0):
addr = "%s:%d" % (host, port)
comm = await connect(listener.contact_address, ssl_context=client_ctx)
assert comm.peer_address == "tls://" + addr
check_tls_extra(comm.extra_info)
await comm.write({"op": "ping", "data": key})
if delay:
await asyncio.sleep(delay)
msg = await comm.read()
assert msg == {"op": "pong", "data": key}
l.append(key)
await comm.close()
await client_communicate(key=1234)
# Many clients at once
N = 100
futures = [client_communicate(key=i, delay=0.05) for i in range(N)]
await asyncio.gather(*futures)
assert set(l) == {1234} | set(range(N))
@pytest.mark.asyncio
async def test_comm_failure_threading():
"""
When we fail to connect, make sure we don't make a lot
of threads.
We only assert for PY3, because the thread limit only is
set for python 3. See github PR #2403 discussion for info.
"""
async def sleep_for_60ms():
max_thread_count = 0
for x in range(60):
await asyncio.sleep(0.001)
thread_count = threading.active_count()
if thread_count > max_thread_count:
max_thread_count = thread_count
return max_thread_count
original_thread_count = threading.active_count()
# tcp.TCPConnector()
sleep_future = sleep_for_60ms()
with pytest.raises(IOError):
await connect("tcp://localhost:28400", 0.052)
max_thread_count = await sleep_future
# 2 is the number set by BaseTCPConnector.executor (ThreadPoolExecutor)
assert max_thread_count <= 2 + original_thread_count
# tcp.TLSConnector()
sleep_future = sleep_for_60ms()
with pytest.raises(IOError):
await connect(
"tls://localhost:28400", 0.052, ssl_context=get_client_ssl_context()
)
max_thread_count = await sleep_future
assert max_thread_count <= 2 + original_thread_count
async def check_inproc_specific(run_client):
"""
Test concrete InProc API.
"""
listener_addr = inproc.global_manager.new_address()
addr_head = listener_addr.rpartition("/")[0]
client_addresses = set()
N_MSGS = 3
async def handle_comm(comm):
assert comm.peer_address.startswith("inproc://" + addr_head)
client_addresses.add(comm.peer_address)
for i in range(N_MSGS):
msg = await comm.read()
msg["op"] = "pong"
await comm.write(msg)
await comm.close()
listener = await inproc.InProcListener(listener_addr, handle_comm)
assert (
listener.listen_address
== listener.contact_address
== "inproc://" + listener_addr
)
l = []
async def client_communicate(key, delay=0):
comm = await connect(listener.contact_address)
assert comm.peer_address == "inproc://" + listener_addr
for i in range(N_MSGS):
await comm.write({"op": "ping", "data": key})
if delay:
await asyncio.sleep(delay)
msg = await comm.read()
assert msg == {"op": "pong", "data": key}
l.append(key)
with pytest.raises(CommClosedError):
await comm.read()
await comm.close()
client_communicate = partial(run_client, client_communicate)
await client_communicate(key=1234)
# Many clients at once
N = 20
futures = [client_communicate(key=i, delay=0.001) for i in range(N)]
await asyncio.gather(*futures)
assert set(l) == {1234} | set(range(N))
assert len(client_addresses) == N + 1
assert listener.contact_address not in client_addresses
def run_coro(func, *args, **kwargs):
return func(*args, **kwargs)
def run_coro_in_thread(func, *args, **kwargs):
fut = Future()
main_loop = ioloop.IOLoop.current()
def run():
thread_loop = ioloop.IOLoop() # need fresh IO loop for run_sync()
try:
res = thread_loop.run_sync(partial(func, *args, **kwargs), timeout=10)
except Exception:
main_loop.add_callback(fut.set_exc_info, sys.exc_info())
else:
main_loop.add_callback(fut.set_result, res)
finally:
thread_loop.close()
t = threading.Thread(target=run)
t.start()
return fut
@pytest.mark.asyncio
async def test_inproc_specific_same_thread():
await check_inproc_specific(run_coro)
@pytest.mark.asyncio
async def test_inproc_specific_different_threads():
await check_inproc_specific(run_coro_in_thread)
#
# Test communications through the abstract API
#
async def check_client_server(
addr,
check_listen_addr=None,
check_contact_addr=None,
listen_args={},
connect_args={},
):
"""
Abstract client / server test.
"""
async def handle_comm(comm):
scheme, loc = parse_address(comm.peer_address)
assert scheme == bound_scheme
msg = await comm.read()
assert msg["op"] == "ping"
msg["op"] = "pong"
await comm.write(msg)
msg = await comm.read()
assert msg["op"] == "foobar"
await comm.close()
# Arbitrary connection args should be ignored
listen_args = listen_args or {"xxx": "bar"}
connect_args = connect_args or {"xxx": "foo"}
listener = await listen(addr, handle_comm, **listen_args)
# Check listener properties
bound_addr = listener.listen_address
bound_scheme, bound_loc = parse_address(bound_addr)
assert bound_scheme in backends
assert bound_scheme == parse_address(addr)[0]
if check_listen_addr is not None:
check_listen_addr(bound_loc)
contact_addr = listener.contact_address
contact_scheme, contact_loc = parse_address(contact_addr)
assert contact_scheme == bound_scheme
if check_contact_addr is not None:
check_contact_addr(contact_loc)
else:
assert contact_addr == bound_addr
# Check client <-> server comms
l = []
async def client_communicate(key, delay=0):
comm = await connect(listener.contact_address, **connect_args)
assert comm.peer_address == listener.contact_address
await comm.write({"op": "ping", "data": key})
await comm.write({"op": "foobar"})
if delay:
await asyncio.sleep(delay)
msg = await comm.read()
assert msg == {"op": "pong", "data": key}
l.append(key)
await comm.close()
await client_communicate(key=1234)
# Many clients at once
futures = [client_communicate(key=i, delay=0.05) for i in range(20)]
await asyncio.gather(*futures)
assert set(l) == {1234} | set(range(20))
listener.stop()
@pytest.mark.asyncio
async def test_ucx_client_server():
pytest.importorskip("distributed.comm.ucx")
ucp = pytest.importorskip("ucp")
addr = ucp.get_address()
await check_client_server("ucx://" + addr)
def tcp_eq(expected_host, expected_port=None):
def checker(loc):
host, port = parse_host_port(loc)
assert host == expected_host
if expected_port is not None:
assert port == expected_port
else:
assert 1023 < port < 65536
return checker
tls_eq = tcp_eq
def inproc_check():
expected_ip = get_ip()
expected_pid = os.getpid()
def checker(loc):
ip, pid, suffix = loc.split("/")
assert ip == expected_ip
assert int(pid) == expected_pid
return checker
@pytest.mark.asyncio
async def test_default_client_server_ipv4():
# Default scheme is (currently) TCP
await check_client_server("127.0.0.1", tcp_eq("127.0.0.1"))
await check_client_server("127.0.0.1:3201", tcp_eq("127.0.0.1", 3201))
await check_client_server("0.0.0.0", tcp_eq("0.0.0.0"), tcp_eq(EXTERNAL_IP4))
await check_client_server(
"0.0.0.0:3202", tcp_eq("0.0.0.0", 3202), tcp_eq(EXTERNAL_IP4, 3202)
)
# IPv4 is preferred for the bound address
await check_client_server("", tcp_eq("0.0.0.0"), tcp_eq(EXTERNAL_IP4))
await check_client_server(
":3203", tcp_eq("0.0.0.0", 3203), tcp_eq(EXTERNAL_IP4, 3203)
)
@requires_ipv6
@pytest.mark.asyncio
async def test_default_client_server_ipv6():
await check_client_server("[::1]", tcp_eq("::1"))
await check_client_server("[::1]:3211", tcp_eq("::1", 3211))
await check_client_server("[::]", tcp_eq("::"), tcp_eq(EXTERNAL_IP6))
await check_client_server(
"[::]:3212", tcp_eq("::", 3212), tcp_eq(EXTERNAL_IP6, 3212)
)
@pytest.mark.asyncio
async def test_tcp_client_server_ipv4():
await check_client_server("tcp://127.0.0.1", tcp_eq("127.0.0.1"))
await check_client_server("tcp://127.0.0.1:3221", tcp_eq("127.0.0.1", 3221))
await check_client_server("tcp://0.0.0.0", tcp_eq("0.0.0.0"), tcp_eq(EXTERNAL_IP4))
await check_client_server(
"tcp://0.0.0.0:3222", tcp_eq("0.0.0.0", 3222), tcp_eq(EXTERNAL_IP4, 3222)
)
await check_client_server("tcp://", tcp_eq("0.0.0.0"), tcp_eq(EXTERNAL_IP4))
await check_client_server(
"tcp://:3223", tcp_eq("0.0.0.0", 3223), tcp_eq(EXTERNAL_IP4, 3223)
)
@requires_ipv6
@pytest.mark.asyncio
async def test_tcp_client_server_ipv6():
await check_client_server("tcp://[::1]", tcp_eq("::1"))
await check_client_server("tcp://[::1]:3231", tcp_eq("::1", 3231))
await check_client_server("tcp://[::]", tcp_eq("::"), tcp_eq(EXTERNAL_IP6))
await check_client_server(
"tcp://[::]:3232", tcp_eq("::", 3232), tcp_eq(EXTERNAL_IP6, 3232)
)
@pytest.mark.asyncio
async def test_tls_client_server_ipv4():
await check_client_server("tls://127.0.0.1", tls_eq("127.0.0.1"), **tls_kwargs)
await check_client_server(
"tls://127.0.0.1:3221", tls_eq("127.0.0.1", 3221), **tls_kwargs
)
await check_client_server(
"tls://", tls_eq("0.0.0.0"), tls_eq(EXTERNAL_IP4), **tls_kwargs
)
@requires_ipv6
@pytest.mark.asyncio
async def test_tls_client_server_ipv6():
await check_client_server("tls://[::1]", tls_eq("::1"), **tls_kwargs)
@pytest.mark.asyncio
async def test_inproc_client_server():
await check_client_server("inproc://", inproc_check())
await check_client_server(inproc.new_address(), inproc_check())
#
# TLS certificate handling
#
@pytest.mark.asyncio
async def test_tls_reject_certificate():
cli_ctx = get_client_ssl_context()
serv_ctx = get_server_ssl_context()
# These certs are not signed by our test CA
bad_cert_key = ("tls-self-signed-cert.pem", "tls-self-signed-key.pem")
bad_cli_ctx = get_client_ssl_context(*bad_cert_key)
bad_serv_ctx = get_server_ssl_context(*bad_cert_key)
async def handle_comm(comm):
scheme, loc = parse_address(comm.peer_address)
assert scheme == "tls"
await comm.close()
# Listener refuses a connector not signed by the CA
listener = await listen("tls://", handle_comm, ssl_context=serv_ctx)
with pytest.raises(EnvironmentError) as excinfo:
comm = await connect(
listener.contact_address, timeout=0.5, ssl_context=bad_cli_ctx
)
await comm.write({"x": "foo"}) # TODO: why is this necessary in Tornado 6 ?
if os.name != "nt":
try:
# See https://serverfault.com/questions/793260/what-does-tlsv1-alert-unknown-ca-mean
# assert "unknown ca" in str(excinfo.value)
pass
except AssertionError:
if os.name == "nt":
assert "An existing connection was forcibly closed" in str(
excinfo.value
)
else:
raise
# Sanity check
comm = await connect(listener.contact_address, timeout=2, ssl_context=cli_ctx)
await comm.close()
# Connector refuses a listener not signed by the CA
listener = await listen("tls://", handle_comm, ssl_context=bad_serv_ctx)
with pytest.raises(EnvironmentError) as excinfo:
await connect(listener.contact_address, timeout=2, ssl_context=cli_ctx)
assert "certificate verify failed" in str(excinfo.value.__cause__)
#
# Test communication closing
#
async def check_comm_closed_implicit(addr, delay=None, listen_args={}, connect_args={}):
async def handle_comm(comm):
await comm.close()
listener = await listen(addr, handle_comm, **listen_args)
comm = await connect(listener.contact_address, **connect_args)
with pytest.raises(CommClosedError):
await comm.write({})
await comm.read()
comm = await connect(listener.contact_address, **connect_args)
with pytest.raises(CommClosedError):
await comm.read()
@pytest.mark.asyncio
async def test_tcp_comm_closed_implicit():
await check_comm_closed_implicit("tcp://127.0.0.1")
@pytest.mark.asyncio
async def test_tls_comm_closed_implicit():
await check_comm_closed_implicit("tls://127.0.0.1", **tls_kwargs)
@pytest.mark.asyncio
async def test_inproc_comm_closed_implicit():
await check_comm_closed_implicit(inproc.new_address())
async def check_comm_closed_explicit(addr, listen_args={}, connect_args={}):
a, b = await get_comm_pair(addr, listen_args=listen_args, connect_args=connect_args)
a_read = a.read()
b_read = b.read()
await a.close()
# In-flight reads should abort with CommClosedError
with pytest.raises(CommClosedError):
await a_read
with pytest.raises(CommClosedError):
await b_read
# New reads as well
with pytest.raises(CommClosedError):
await a.read()
with pytest.raises(CommClosedError):
await b.read()
# And writes
with pytest.raises(CommClosedError):
await a.write({})
with pytest.raises(CommClosedError):
await b.write({})
await b.close()
@pytest.mark.asyncio
async def test_tcp_comm_closed_explicit():
await check_comm_closed_explicit("tcp://127.0.0.1")
@pytest.mark.asyncio
async def test_tls_comm_closed_explicit():
await check_comm_closed_explicit("tls://127.0.0.1", **tls_kwargs)
@pytest.mark.asyncio
async def test_inproc_comm_closed_explicit():
await check_comm_closed_explicit(inproc.new_address())
@pytest.mark.asyncio
async def test_inproc_comm_closed_explicit_2():
listener_errors = []
async def handle_comm(comm):
# Wait
try:
await comm.read()
except CommClosedError:
assert comm.closed()
listener_errors.append(True)
else:
await comm.close()
listener = await listen("inproc://", handle_comm)
comm = await connect(listener.contact_address)
await comm.close()
assert comm.closed()
start = time()
while len(listener_errors) < 1:
assert time() < start + 1
await asyncio.sleep(0.01)
assert len(listener_errors) == 1
with pytest.raises(CommClosedError):
await comm.read()
with pytest.raises(CommClosedError):
await comm.write("foo")
comm = await connect(listener.contact_address)
await comm.write("foo")
with pytest.raises(CommClosedError):
await comm.read()
with pytest.raises(CommClosedError):
await comm.write("foo")
assert comm.closed()
comm = await connect(listener.contact_address)
await comm.write("foo")
start = time()
while not comm.closed():
await asyncio.sleep(0.01)
assert time() < start + 2
await comm.close()
await comm.close()
#
# Various stress tests
#
async def echo(comm):
message = await comm.read()
await comm.write(message)
@pytest.mark.asyncio
async def test_retry_connect(monkeypatch):
async def echo(comm):
message = await comm.read()
await comm.write(message)
class UnreliableConnector(TCPConnector):
def __init__(self):
self.num_failures = 2
self.failures = 0
super().__init__()
async def connect(self, address, deserialize=True, **connection_args):
if self.failures > self.num_failures:
return await super().connect(address, deserialize, **connection_args)
else:
self.failures += 1
raise IOError()
class UnreliableBackend(TCPBackend):
_connector_class = UnreliableConnector
monkeypatch.setitem(backends, "tcp", UnreliableBackend())
listener = await listen("tcp://127.0.0.1:1234", echo)
try:
comm = await connect(listener.contact_address)
await comm.write(b"test")
msg = await comm.read()
assert msg == b"test"
finally:
listener.stop()
@pytest.mark.asyncio
async def test_handshake_slow_comm(monkeypatch):
class SlowComm(TCP):
def __init__(self, *args, delay_in_comm=0.5, **kwargs):
super().__init__(*args, **kwargs)
self.delay_in_comm = delay_in_comm
async def read(self, *args, **kwargs):
await asyncio.sleep(self.delay_in_comm)
return await super().read(*args, **kwargs)
async def write(self, *args, **kwargs):
await asyncio.sleep(self.delay_in_comm)
res = await super(type(self), self).write(*args, **kwargs)
return res
class SlowConnector(TCPConnector):
comm_class = SlowComm
class SlowBackend(TCPBackend):
_connector_class = SlowConnector
monkeypatch.setitem(backends, "tcp", SlowBackend())
listener = await listen("tcp://127.0.0.1:1234", echo)
try:
comm = await connect(listener.contact_address)
await comm.write(b"test")
msg = await comm.read()
assert msg == b"test"
import dask
with dask.config.set({"distributed.comm.timeouts.connect": "100ms"}):
with pytest.raises(
IOError, match="Timed out during handshake while connecting to"
):
await connect(listener.contact_address)
finally:
listener.stop()
async def check_connect_timeout(addr):
t1 = time()
with pytest.raises(IOError):
await connect(addr, timeout=0.15)
dt = time() - t1
assert 1 >= dt >= 0.1
@pytest.mark.asyncio
async def test_tcp_connect_timeout():
await check_connect_timeout("tcp://127.0.0.1:44444")
@pytest.mark.asyncio
async def test_inproc_connect_timeout():
await check_connect_timeout(inproc.new_address())
async def check_many_listeners(addr):
async def handle_comm(comm):
pass
listeners = []
N = 100
for i in range(N):
listener = await listen(addr, handle_comm)
listeners.append(listener)
assert len(set(l.listen_address for l in listeners)) == N
assert len(set(l.contact_address for l in listeners)) == N
for listener in listeners:
listener.stop()
@pytest.mark.asyncio
async def test_tcp_many_listeners():
await check_many_listeners("tcp://127.0.0.1")
await check_many_listeners("tcp://0.0.0.0")
await check_many_listeners("tcp://")
@pytest.mark.asyncio
async def test_inproc_many_listeners():
await check_many_listeners("inproc://")
#
# Test deserialization
#
async def check_listener_deserialize(addr, deserialize, in_value, check_out):
q = asyncio.Queue()
async def handle_comm(comm):
msg = await comm.read()
q.put_nowait(msg)
await comm.close()
async with listen(addr, handle_comm, deserialize=deserialize) as listener:
comm = await connect(listener.contact_address)
await comm.write(in_value)
out_value = await q.get()
check_out(out_value)
await comm.close()
async def check_connector_deserialize(addr, deserialize, in_value, check_out):
done = asyncio.Event()
async def handle_comm(comm):
await comm.write(in_value)
await done.wait()
await comm.close()
async with listen(addr, handle_comm) as listener:
comm = await connect(listener.contact_address, deserialize=deserialize)
out_value = await comm.read()
done.set()
await comm.close()
check_out(out_value)
async def check_deserialize(addr):
"""
Check the "deserialize" flag on connect() and listen().
"""
# Test with Serialize and Serialized objects
msg = {
"op": "update",
"x": b"abc",
"to_ser": [to_serialize(123)],
"ser": Serialized(*serialize(456)),
}
msg_orig = msg.copy()
def check_out_false(out_value):
# Check output with deserialize=False
out_value = out_value.copy() # in case transport passed the object as-is
to_ser = out_value.pop("to_ser")
ser = out_value.pop("ser")
expected_msg = msg_orig.copy()
del expected_msg["ser"]
del expected_msg["to_ser"]
assert out_value == expected_msg
assert isinstance(ser, Serialized)
assert deserialize(ser.header, ser.frames) == 456
assert isinstance(to_ser, list)
(to_ser,) = to_ser
# The to_serialize() value could have been actually serialized
# or not (it's a transport-specific optimization)
if isinstance(to_ser, Serialized):
assert deserialize(to_ser.header, to_ser.frames) == 123
else:
assert to_ser == to_serialize(123)
def check_out_true(out_value):
# Check output with deserialize=True
expected_msg = msg.copy()
expected_msg["ser"] = 456
expected_msg["to_ser"] = [123]
assert out_value == expected_msg
await check_listener_deserialize(addr, False, msg, check_out_false)
await check_connector_deserialize(addr, False, msg, check_out_false)
await check_listener_deserialize(addr, True, msg, check_out_true)
await check_connector_deserialize(addr, True, msg, check_out_true)
# Test with long bytestrings, large enough to be transferred
# as a separate payload
_uncompressible = os.urandom(1024 ** 2) * 4 # end size: 8 MB
msg = {
"op": "update",
"x": _uncompressible,
"to_ser": [to_serialize(_uncompressible)],
"ser": Serialized(*serialize(_uncompressible)),
}
msg_orig = msg.copy()
def check_out(deserialize_flag, out_value):
# Check output with deserialize=False
assert sorted(out_value) == sorted(msg_orig)
out_value = out_value.copy() # in case transport passed the object as-is
to_ser = out_value.pop("to_ser")
ser = out_value.pop("ser")
expected_msg = msg_orig.copy()
del expected_msg["ser"]
del expected_msg["to_ser"]
assert out_value == expected_msg
if deserialize_flag:
assert isinstance(ser, (bytes, bytearray))
assert bytes(ser) == _uncompressible
else:
assert isinstance(ser, Serialized)
assert deserialize(ser.header, ser.frames) == _uncompressible
assert isinstance(to_ser, list)
(to_ser,) = to_ser
# The to_serialize() value could have been actually serialized
# or not (it's a transport-specific optimization)
if isinstance(to_ser, Serialized):
assert deserialize(to_ser.header, to_ser.frames) == _uncompressible
else:
assert to_ser == to_serialize(_uncompressible)
await check_listener_deserialize(addr, False, msg, partial(check_out, False))
await check_connector_deserialize(addr, False, msg, partial(check_out, False))
await check_listener_deserialize(addr, True, msg, partial(check_out, True))
await check_connector_deserialize(addr, True, msg, partial(check_out, True))
@pytest.mark.xfail(reason="intermittent failure on windows")
@pytest.mark.asyncio
async def test_tcp_deserialize():
await check_deserialize("tcp://")
@pytest.mark.asyncio
async def test_inproc_deserialize():
await check_deserialize("inproc://")
async def check_deserialize_roundtrip(addr):
"""
Sanity check round-tripping with "deserialize" on and off.
"""
# Test with long bytestrings, large enough to be transferred
# as a separate payload
_uncompressible = os.urandom(1024 ** 2) * 4 # end size: 4 MB
msg = {
"op": "update",
"x": _uncompressible,
"to_ser": [to_serialize(_uncompressible)],
"ser": Serialized(*serialize(_uncompressible)),
}
for should_deserialize in (True, False):
a, b = await get_comm_pair(addr, deserialize=should_deserialize)
await a.write(msg)
got = await b.read()
await b.write(got)
got = await a.read()
assert sorted(got) == sorted(msg)
for k in ("op", "x"):
assert got[k] == msg[k]
if should_deserialize:
assert isinstance(got["to_ser"][0], (bytes, bytearray))
assert isinstance(got["ser"], (bytes, bytearray))
else:
assert isinstance(got["to_ser"][0], (to_serialize, Serialized))
assert isinstance(got["ser"], Serialized)
@pytest.mark.asyncio
async def test_inproc_deserialize_roundtrip():
await check_deserialize_roundtrip("inproc://")
@pytest.mark.asyncio
async def test_tcp_deserialize_roundtrip():
await check_deserialize_roundtrip("tcp://")
def _raise_eoferror():
raise EOFError
class _EOFRaising:
def __reduce__(self):
return _raise_eoferror, ()
async def check_deserialize_eoferror(addr):
"""
EOFError when deserializing should close the comm.
"""
async def handle_comm(comm):
await comm.write({"data": to_serialize(_EOFRaising())})
with pytest.raises(CommClosedError):
await comm.read()
async with listen(addr, handle_comm) as listener:
comm = await connect(listener.contact_address, deserialize=deserialize)
with pytest.raises(CommClosedError):
await comm.read()
@pytest.mark.asyncio
async def test_tcp_deserialize_eoferror():
await check_deserialize_eoferror("tcp://")
#
# Test various properties
#
async def check_repr(a, b):
assert "closed" not in repr(a)
assert "closed" not in repr(b)
await a.close()
assert "closed" in repr(a)
await b.close()
assert "closed" in repr(b)
@pytest.mark.asyncio
async def test_tcp_repr():
a, b = await get_tcp_comm_pair()
assert a.local_address in repr(b)
assert b.local_address in repr(a)
await check_repr(a, b)
@pytest.mark.asyncio
async def test_tls_repr():
a, b = await get_tls_comm_pair()
assert a.local_address in repr(b)
assert b.local_address in repr(a)
await check_repr(a, b)
@pytest.mark.asyncio
async def test_inproc_repr():
a, b = await get_inproc_comm_pair()
assert a.local_address in repr(b)
assert b.local_address in repr(a)
await check_repr(a, b)
async def check_addresses(a, b):
assert a.peer_address == b.local_address
assert a.local_address == b.peer_address
a.abort()
b.abort()
@pytest.mark.asyncio
async def test_tcp_adresses():
a, b = await get_tcp_comm_pair()
await check_addresses(a, b)
@pytest.mark.asyncio
async def test_tls_adresses():
a, b = await get_tls_comm_pair()
await check_addresses(a, b)
@pytest.mark.asyncio
async def test_inproc_adresses():
a, b = await get_inproc_comm_pair()
await check_addresses(a, b)
def test_register_backend_entrypoint():
# Code adapted from pandas backend entry point testing
# https://github.com/pandas-dev/pandas/blob/2470690b9f0826a8feb426927694fa3500c3e8d2/pandas/tests/plotting/test_backend.py#L50-L76
dist = pkg_resources.get_distribution("distributed")
if dist.module_path not in distributed.__file__:
# We are running from a non-installed distributed, and this test is invalid
pytest.skip("Testing a non-installed distributed")
mod = types.ModuleType("dask_udp")
mod.UDPBackend = lambda: 1
sys.modules[mod.__name__] = mod
entry_point_name = "distributed.comm.backends"
backends_entry_map = pkg_resources.get_entry_map("distributed")
if entry_point_name not in backends_entry_map:
backends_entry_map[entry_point_name] = dict()
backends_entry_map[entry_point_name]["udp"] = pkg_resources.EntryPoint(
"udp", mod.__name__, attrs=["UDPBackend"], dist=dist
)
result = get_backend("udp")
assert result == 1
|