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 | from distributed.core import ConnectionPool
from distributed.comm import Comm
from distributed.utils_test import gen_cluster, loop # noqa: F401
from distributed.utils_comm import pack_data, subs_multiple, gather_from_workers, retry
from unittest import mock
import pytest
def test_pack_data():
data = {"x": 1}
assert pack_data(("x", "y"), data) == (1, "y")
assert pack_data({"a": "x", "b": "y"}, data) == {"a": 1, "b": "y"}
assert pack_data({"a": ["x"], "b": "y"}, data) == {"a": [1], "b": "y"}
def test_subs_multiple():
data = {"x": 1, "y": 2}
assert subs_multiple((sum, [0, "x", "y", "z"]), data) == (sum, [0, 1, 2, "z"])
assert subs_multiple((sum, [0, ["x", "y", "z"]]), data) == (sum, [0, [1, 2, "z"]])
dsk = {"a": (sum, ["x", "y"])}
assert subs_multiple(dsk, data) == {"a": (sum, [1, 2])}
# Tuple key
data = {"x": 1, ("y", 0): 2}
dsk = {"a": (sum, ["x", ("y", 0)])}
assert subs_multiple(dsk, data) == {"a": (sum, [1, 2])}
@gen_cluster(client=True)
async def test_gather_from_workers_permissive(c, s, a, b):
rpc = await ConnectionPool()
x = await c.scatter({"x": 1}, workers=a.address)
data, missing, bad_workers = await gather_from_workers(
{"x": [a.address], "y": [b.address]}, rpc=rpc
)
assert data == {"x": 1}
assert list(missing) == ["y"]
class BrokenComm(Comm):
peer_address = None
local_address = None
def close(self):
pass
def closed(self):
pass
def abort(self):
pass
def read(self, deserializers=None):
raise EnvironmentError
def write(self, msg, serializers=None, on_error=None):
raise EnvironmentError
class BrokenConnectionPool(ConnectionPool):
async def connect(self, *args, **kwargs):
return BrokenComm()
@gen_cluster(client=True)
async def test_gather_from_workers_permissive_flaky(c, s, a, b):
x = await c.scatter({"x": 1}, workers=a.address)
rpc = await BrokenConnectionPool()
data, missing, bad_workers = await gather_from_workers({"x": [a.address]}, rpc=rpc)
assert missing == {"x": [a.address]}
assert bad_workers == [a.address]
def test_retry_no_exception(loop):
n_calls = 0
retval = object()
async def coro():
nonlocal n_calls
n_calls += 1
return retval
assert (
loop.run_sync(lambda: retry(coro, count=0, delay_min=-1, delay_max=-1))
is retval
)
assert n_calls == 1
def test_retry0_raises_immediately(loop):
# test that using max_reties=0 raises after 1 call
n_calls = 0
async def coro():
nonlocal n_calls
n_calls += 1
raise RuntimeError(f"RT_ERROR {n_calls}")
with pytest.raises(RuntimeError, match="RT_ERROR 1"):
loop.run_sync(lambda: retry(coro, count=0, delay_min=-1, delay_max=-1))
assert n_calls == 1
def test_retry_does_retry_and_sleep(loop):
# test the retry and sleep pattern of `retry`
n_calls = 0
class MyEx(Exception):
pass
async def coro():
nonlocal n_calls
n_calls += 1
raise MyEx(f"RT_ERROR {n_calls}")
sleep_calls = []
async def my_sleep(amount):
sleep_calls.append(amount)
return
with mock.patch("asyncio.sleep", my_sleep):
with pytest.raises(MyEx, match="RT_ERROR 6"):
loop.run_sync(
lambda: retry(
coro,
retry_on_exceptions=(MyEx,),
count=5,
delay_min=1.0,
delay_max=6.0,
jitter_fraction=0.0,
)
)
assert n_calls == 6
assert sleep_calls == [0.0, 1.0, 3.0, 6.0, 6.0]
|