-
-
Notifications
You must be signed in to change notification settings - Fork 710
/
test_cancelled_state.py
454 lines (368 loc) · 14.5 KB
/
test_cancelled_state.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
import asyncio
import distributed
from distributed import Event, Lock, Worker
from distributed.client import wait
from distributed.utils_test import (
_LockedCommPool,
assert_story,
gen_cluster,
inc,
slowinc,
)
async def wait_for_state(key, state, dask_worker):
while key not in dask_worker.tasks or dask_worker.tasks[key].state != state:
await asyncio.sleep(0.005)
async def wait_for_cancelled(key, dask_worker):
while key in dask_worker.tasks:
if dask_worker.tasks[key].state == "cancelled":
return
await asyncio.sleep(0.005)
assert False
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_abort_execution_release(c, s, a):
fut = c.submit(slowinc, 1, delay=1)
await wait_for_state(fut.key, "executing", a)
fut.release()
await wait_for_cancelled(fut.key, a)
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_abort_execution_reschedule(c, s, a):
fut = c.submit(slowinc, 1, delay=1)
await wait_for_state(fut.key, "executing", a)
fut.release()
await wait_for_cancelled(fut.key, a)
fut = c.submit(slowinc, 1, delay=0.1)
await fut
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_abort_execution_add_as_dependency(c, s, a):
fut = c.submit(slowinc, 1, delay=1)
await wait_for_state(fut.key, "executing", a)
fut.release()
await wait_for_cancelled(fut.key, a)
fut = c.submit(slowinc, 1, delay=1)
fut = c.submit(slowinc, fut, delay=1)
await fut
@gen_cluster(client=True)
async def test_abort_execution_to_fetch(c, s, a, b):
ev = Event()
def f(ev):
ev.wait()
return 123
fut = c.submit(f, ev, key="f1", workers=[a.worker_address])
await wait_for_state(fut.key, "executing", a)
fut.release()
await wait_for_cancelled(fut.key, a)
# While the first worker is still trying to compute f1, we'll resubmit it to
# another worker with a smaller delay. The key is still the same
fut = c.submit(inc, 1, key="f1", workers=[b.worker_address])
# then, a must switch the execute to fetch. Instead of doing so, it will
# simply re-use the currently computing result.
fut = c.submit(inc, fut, workers=[a.worker_address], key="f2")
await wait_for_state("f2", "waiting", a)
await ev.set()
assert await fut == 124 # It would be 3 if the result was copied from b
del fut
while "f1" in a.tasks:
await asyncio.sleep(0.01)
assert_story(
a.story("f1"),
[
("f1", "compute-task", "released"),
("f1", "released", "waiting", "waiting", {"f1": "ready"}),
("f1", "waiting", "ready", "ready", {"f1": "executing"}),
("f1", "ready", "executing", "executing", {}),
("free-keys", ("f1",)),
("f1", "executing", "released", "cancelled", {}),
("f1", "cancelled", "fetch", "resumed", {}),
("f1", "resumed", "memory", "memory", {"f2": "ready"}),
("free-keys", ("f1",)),
("f1", "memory", "released", "released", {}),
("f1", "released", "forgotten", "forgotten", {}),
],
)
@gen_cluster(client=True)
async def test_worker_find_missing(c, s, a, b):
fut = c.submit(inc, 1, workers=[a.address])
await fut
# We do not want to use proper API since it would ensure that the cluster is
# informed properly
del a.data[fut.key]
del a.tasks[fut.key]
# Actually no worker has the data; the scheduler is supposed to reschedule
assert await c.submit(inc, fut, workers=[b.address]) == 3
@gen_cluster(client=True)
async def test_worker_stream_died_during_comm(c, s, a, b):
write_queue = asyncio.Queue()
write_event = asyncio.Event()
b.rpc = _LockedCommPool(
b.rpc,
write_queue=write_queue,
write_event=write_event,
)
fut = c.submit(inc, 1, workers=[a.address], allow_other_workers=True)
await fut
# Actually no worker has the data; the scheduler is supposed to reschedule
res = c.submit(inc, fut, workers=[b.address])
await write_queue.get()
await a.close()
write_event.set()
await res
assert any("receive-dep-failed" in msg for msg in b.log)
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_flight_to_executing_via_cancelled_resumed(c, s, b):
block_get_data = asyncio.Lock()
block_compute = Lock()
enter_get_data = asyncio.Event()
await block_get_data.acquire()
class BrokenWorker(Worker):
async def get_data(self, comm, *args, **kwargs):
enter_get_data.set()
async with block_get_data:
comm.abort()
def blockable_compute(x, lock):
with lock:
return x + 1
async with BrokenWorker(s.address) as a:
await c.wait_for_workers(2)
fut1 = c.submit(
blockable_compute,
1,
lock=block_compute,
workers=[a.address],
allow_other_workers=True,
key="fut1",
)
fut2 = c.submit(inc, fut1, workers=[b.address], key="fut2")
await enter_get_data.wait()
await block_compute.acquire()
# Close in scheduler to ensure we transition and reschedule task properly
await s.close_worker(worker=a.address, stimulus_id="test")
await wait_for_state(fut1.key, "resumed", b)
block_get_data.release()
await block_compute.release()
assert await fut2 == 3
b_story = b.story(fut1.key)
assert any("receive-dep-failed" in msg for msg in b_story)
assert any("cancelled" in msg for msg in b_story)
assert any("resumed" in msg for msg in b_story)
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_executing_cancelled_error(c, s, w):
"""One worker with one thread. We provoke an executing->cancelled transition
and let the task err. This test ensures that there is no residual state
(e.g. a semaphore) left blocking the thread"""
lock = distributed.Lock()
await lock.acquire()
async def wait_and_raise(*args, **kwargs):
async with lock:
raise RuntimeError()
fut = c.submit(wait_and_raise)
await wait_for_state(fut.key, "executing", w)
# Queue up another task to ensure this is not affected by our error handling
fut2 = c.submit(inc, 1)
await wait_for_state(fut2.key, "ready", w)
fut.release()
await wait_for_state(fut.key, "cancelled", w)
await lock.release()
# At this point we do not fetch the result of the future since the future
# itself would raise a cancelled exception. At this point we're concerned
# about the worker. The task should transition over error to be eventually
# forgotten since we no longer hold a ref.
while fut.key in w.tasks:
await asyncio.sleep(0.01)
assert await fut2 == 2
# Everything should still be executing as usual after this
await c.submit(sum, c.map(inc, range(10))) == sum(map(inc, range(10)))
# Everything above this line should be generically true, regardless of
# refactoring. Below verifies some implementation specific test assumptions
story = w.story(fut.key)
start_finish = [(msg[1], msg[2], msg[3]) for msg in story if len(msg) == 7]
assert ("executing", "released", "cancelled") in start_finish
assert ("cancelled", "error", "error") in start_finish
assert ("error", "released", "released") in start_finish
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_flight_cancelled_error(c, s, b):
"""One worker with one thread. We provoke an flight->cancelled transition
and let the task err."""
lock = asyncio.Lock()
await lock.acquire()
class BrokenWorker(Worker):
block_get_data = True
async def get_data(self, comm, *args, **kwargs):
if self.block_get_data:
async with lock:
comm.abort()
return await super().get_data(comm, *args, **kwargs)
async with BrokenWorker(s.address) as a:
await c.wait_for_workers(2)
fut1 = c.submit(inc, 1, workers=[a.address], allow_other_workers=True)
fut2 = c.submit(inc, fut1, workers=[b.address])
await wait_for_state(fut1.key, "flight", b)
fut2.release()
fut1.release()
await wait_for_state(fut1.key, "cancelled", b)
lock.release()
# At this point we do not fetch the result of the future since the
# future itself would raise a cancelled exception. At this point we're
# concerned about the worker. The task should transition over error to
# be eventually forgotten since we no longer hold a ref.
while fut1.key in b.tasks:
await asyncio.sleep(0.01)
a.block_get_data = False
# Everything should still be executing as usual after this
assert await c.submit(sum, c.map(inc, range(10))) == sum(map(inc, range(10)))
@gen_cluster(client=True, nthreads=[("", 1)])
async def test_in_flight_lost_after_resumed(c, s, b):
block_get_data = asyncio.Lock()
in_get_data = asyncio.Event()
await block_get_data.acquire()
lock_executing = Lock()
def block_execution(lock):
with lock:
return 1
class BlockedGetData(Worker):
async def get_data(self, comm, *args, **kwargs):
in_get_data.set()
async with block_get_data:
return await super().get_data(comm, *args, **kwargs)
async with BlockedGetData(s.address, name="blocked-get-dataworker") as a:
fut1 = c.submit(
block_execution,
lock_executing,
workers=[a.address],
key="fut1",
)
# Ensure fut1 is in memory but block any further execution afterwards to
# ensure we control when the recomputation happens
await wait(fut1)
await lock_executing.acquire()
fut2 = c.submit(inc, fut1, workers=[b.address], key="fut2")
# This ensures that B already fetches the task, i.e. after this the task
# is guaranteed to be in flight
await in_get_data.wait()
assert fut1.key in b.tasks
assert b.tasks[fut1.key].state == "flight"
s.set_restrictions({fut1.key: [a.address, b.address]})
# It is removed, i.e. get_data is guaranteed to fail and f1 is scheduled
# to be recomputed on B
await s.remove_worker(a.address, "foo", close=False, safe=True)
while not b.tasks[fut1.key].state == "resumed":
await asyncio.sleep(0.01)
fut1.release()
fut2.release()
while not b.tasks[fut1.key].state == "cancelled":
await asyncio.sleep(0.01)
block_get_data.release()
while b.tasks:
await asyncio.sleep(0.01)
assert_story(
b.story(fut1.key),
expect=[
# The initial free-keys is rejected
("free-keys", (fut1.key,)),
(fut1.key, "resumed", "released", "cancelled", {}),
# After gather_dep receives the data, the task is forgotten
(fut1.key, "cancelled", "memory", "released", {fut1.key: "forgotten"}),
],
)
@gen_cluster(client=True)
async def test_cancelled_error(c, s, a, b):
executing = Event()
lock_executing = Lock()
await lock_executing.acquire()
def block_execution(event, lock):
event.set()
with lock:
raise RuntimeError()
fut1 = c.submit(
block_execution,
executing,
lock_executing,
workers=[b.address],
allow_other_workers=True,
key="fut1",
)
await executing.wait()
assert b.tasks[fut1.key].state == "executing"
fut1.release()
while b.tasks[fut1.key].state == "executing":
await asyncio.sleep(0.01)
await lock_executing.release()
while b.tasks:
await asyncio.sleep(0.01)
assert_story(
b.story(fut1.key),
[
(fut1.key, "executing", "released", "cancelled", {}),
(fut1.key, "cancelled", "error", "error", {fut1.key: "released"}),
],
)
@gen_cluster(client=True, nthreads=[("", 1, {"resources": {"A": 1}})])
async def test_cancelled_error_with_resources(c, s, a):
executing = Event()
lock_executing = Lock()
await lock_executing.acquire()
def block_execution(event, lock):
event.set()
with lock:
raise RuntimeError()
fut1 = c.submit(
block_execution,
executing,
lock_executing,
resources={"A": 1},
key="fut1",
)
await executing.wait()
fut2 = c.submit(inc, 1, resources={"A": 1}, key="fut2")
while fut2.key not in a.tasks:
await asyncio.sleep(0.01)
fut1.release()
while a.tasks[fut1.key].state == "executing":
await asyncio.sleep(0.01)
await lock_executing.release()
assert await fut2 == 2
@gen_cluster(client=True, nthreads=[("", 1)] * 2)
async def test_cancelled_resumed_after_flight_with_dependencies(c, s, w2, w3):
# See https://github.com/dask/distributed/pull/6327#discussion_r872231090
block_get_data_1 = asyncio.Lock()
enter_get_data_1 = asyncio.Event()
await block_get_data_1.acquire()
class BlockGetDataWorker(Worker):
def __init__(self, *args, get_data_event, get_data_lock, **kwargs):
self._get_data_event = get_data_event
self._get_data_lock = get_data_lock
super().__init__(*args, **kwargs)
async def get_data(self, comm, *args, **kwargs):
self._get_data_event.set()
async with self._get_data_lock:
return await super().get_data(comm, *args, **kwargs)
async with await BlockGetDataWorker(
s.address,
get_data_event=enter_get_data_1,
get_data_lock=block_get_data_1,
name="w1",
) as w1:
f1 = c.submit(inc, 1, key="f1", workers=[w1.address])
f2 = c.submit(inc, 2, key="f2", workers=[w1.address])
f3 = c.submit(sum, [f1, f2], key="f3", workers=[w1.address])
await wait(f3)
f4 = c.submit(inc, f3, key="f4", workers=[w2.address])
await enter_get_data_1.wait()
s.set_restrictions(
{
f1.key: {w3.address},
f2.key: {w3.address},
f3.key: {w2.address},
}
)
await s.remove_worker(w1.address, "stim-id")
await wait_for_state(f3.key, "resumed", w2)
assert_story(
w2.log,
[
(f3.key, "flight", "released", "cancelled", {}),
# ...
(f3.key, "cancelled", "waiting", "resumed", {}),
],
)
# w1 closed
assert await f4 == 6