-
-
Notifications
You must be signed in to change notification settings - Fork 749
Closed
Labels
flaky testIntermittent failures on CI.Intermittent failures on CI.
Description
See the test report, looks like this started failing on 2022-06-06 and all failures are the same (2 elements in the story instead of 3)
____________________ test_gather_dep_one_worker_always_busy ____________________
c = <Client: No scheduler connected>
s = <Scheduler 'tcp://127.0.0.1:62880', workers: 0, cores: 0, tasks: 0>
a = <Worker 'tcp://127.0.0.1:62881', name: 0, status: closed, stored: 0, running: 0/1, ready: 0, comm: 0, waiting: 0>
b = <Worker 'tcp://127.0.0.1:62883', name: 1, status: closed, stored: 0, running: 0/2, ready: 0, comm: 0, waiting: 1>
@gen_cluster(client=True)
asyncdeftest_gather_dep_one_worker_always_busy(c, s, a, b):
# Ensure that both dependencies for H are on another worker than H itself.
# The worker where the dependencies are on is then later blocked such that
# the data cannot be fetched
# In the past it was important that there is more than one key on the
# worker. This should be kept to avoid any edge case specific to one
f = c.submit(inc, 1, key="f", workers=[a.address])
g = c.submit(inc, 2, key="g", workers=[a.address])
await wait([f, g])
assertset(a.tasks) == {"f", "g"}
# We will block A for any outgoing communication. This simulates an
# overloaded worker which will always return "busy" for get_data requests,
# effectively blocking H indefinitely
a.outgoing_current_count = 10000000
h = c.submit(add, f, g, key="h", workers=[b.address])
while h.key notin b.tasks:
await asyncio.sleep(0.01)
assert b.tasks[h.key].state == "waiting"
assert b.tasks[f.key].state in ("flight", "fetch")
assert b.tasks[g.key].state in ("flight", "fetch")
with pytest.raises(asyncio.TimeoutError):
await h.result(timeout=0.5)
story = b.story("busy-gather")
# 1 busy response straight away, followed by 1 retry every 150ms for 500ms.
# The requests for b and g are clustered together in single messages.
> assert3 <= len(story) <= 7
E AssertionError: assert 3 <= 2
E + where 2 = len([('busy-gather', 'tcp://127.0.0.1:62881', {'f', 'g'}, 'compute-task-1654705251.4192011', 1654705251.4256918), ('busy-gather', 'tcp://127.0.0.1:62881', {'f', 'g'}, 'retry-busy-worker-1654705251.704484', 1654705251.7067032)])
distributed/tests/test_worker.py:1995: AssertionError
----------------------------- Captured stdout call -----------------------------
Dumped cluster state to test_cluster_dump/test_gather_dep_one_worker_always_busy.yaml
----------------------------- Captured stderr call -----------------------------
2022-06-08 16:20:51,280 - distributed.scheduler - INFO - State start
2022-06-08 16:20:51,283 - distributed.scheduler - INFO - Clear task state
2022-06-08 16:20:51,283 - distributed.scheduler - INFO - Scheduler at: tcp://127.0.0.1:62880
2022-06-08 16:20:51,283 - distributed.scheduler - INFO - dashboard at: 127.0.0.1:62879
2022-06-08 16:20:51,292 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:62881
2022-06-08 16:20:51,292 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:62881
2022-06-08 16:20:51,293 - distributed.worker - INFO - dashboard at: 127.0.0.1:62882
2022-06-08 16:20:51,293 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:62880
2022-06-08 16:20:51,293 - distributed.worker - INFO - -------------------------------------------------
2022-06-08 16:20:51,293 - distributed.worker - INFO - Threads: 1
2022-06-08 16:20:51,293 - distributed.worker - INFO - Memory: 14.00 GiB
2022-06-08 16:20:51,293 - distributed.worker - INFO - Local Directory: /var/folders/24/8k48jl6d249_n_qfxwsl6xvm0000gn/T/tmppzbkwlwu/dask-worker-space/worker-7yj3fim4
2022-06-08 16:20:51,293 - distributed.worker - INFO - -------------------------------------------------
2022-06-08 16:20:51,294 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:62883
2022-06-08 16:20:51,295 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:62883
2022-06-08 16:20:51,295 - distributed.worker - INFO - dashboard at: 127.0.0.1:62884
2022-06-08 16:20:51,295 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:62880
2022-06-08 16:20:51,295 - distributed.worker - INFO - -------------------------------------------------
2022-06-08 16:20:51,295 - distributed.worker - INFO - Threads: 2
2022-06-08 16:20:51,295 - distributed.worker - INFO - Memory: 14.00 GiB
2022-06-08 16:20:51,295 - distributed.worker - INFO - Local Directory: /var/folders/24/8k48jl6d249_n_qfxwsl6xvm0000gn/T/tmppzbkwlwu/dask-worker-space/worker-5yd64ba_
2022-06-08 16:20:51,296 - distributed.worker - INFO - -------------------------------------------------
2022-06-08 16:20:51,304 - distributed.scheduler - INFO - Register worker <WorkerState 'tcp://127.0.0.1:62881', name: 0, status: init, memory: 0, processing: 0>
2022-06-08 16:20:51,305 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:62881
2022-06-08 16:20:51,305 - distributed.core - INFO - Starting established connection
2022-06-08 16:20:51,306 - distributed.scheduler - INFO - Register worker <WorkerState 'tcp://127.0.0.1:62883', name: 1, status: init, memory: 0, processing: 0>
2022-06-08 16:20:51,307 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:62883
2022-06-08 16:20:51,307 - distributed.core - INFO - Starting established connection
2022-06-08 16:20:51,307 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:62880
2022-06-08 16:20:51,307 - distributed.worker - INFO - -------------------------------------------------
2022-06-08 16:20:51,308 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:62880
2022-06-08 16:20:51,308 - distributed.worker - INFO - -------------------------------------------------
2022-06-08 16:20:51,308 - distributed.core - INFO - Starting established connection
2022-06-08 16:20:51,309 - distributed.core - INFO - Starting established connection
2022-06-08 16:20:51,370 - distributed.scheduler - INFO - Receive client connection: Client-f6a8ab48-e746-11ec-9190-0050568893d8
2022-06-08 16:20:51,371 - distributed.core - INFO - Starting established connection
2022-06-08 16:20:52,269 - distributed.scheduler - INFO - Remove client Client-f6a8ab48-e746-11ec-9190-0050568893d8
2022-06-08 16:20:52,270 - distributed.scheduler - INFO - Remove client Client-f6a8ab48-e746-11ec-9190-0050568893d8
2022-06-08 16:20:52,271 - distributed.scheduler - INFO - Close client connection: Client-f6a8ab48-e746-11ec-9190-0050568893d8
2022-06-08 16:20:52,274 - distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:62881
2022-06-08 16:20:52,276 - distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:62883
2022-06-08 16:20:52,280 - distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://127.0.0.1:62881', name: 0, status: closing, memory: 0, processing: 0>
2022-06-08 16:20:52,280 - distributed.core - INFO - Removing comms to tcp://127.0.0.1:62881
2022-06-08 16:20:52,280 - distributed.worker - INFO - Connection to scheduler broken. Closing without reporting. ID: Worker-750338e4-7088-470b-880c-655d0a63f80c Address tcp://127.0.0.1:62881 Status: Status.closing
2022-06-08 16:20:52,282 - distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://127.0.0.1:62883', name: 1, status: closing, memory: 0, processing: 0>
2022-06-08 16:20:52,282 - distributed.core - INFO - Removing comms to tcp://127.0.0.1:62883
2022-06-08 16:20:52,282 - distributed.scheduler - INFO - Lost all workers
2022-06-08 16:20:52,283 - distributed.worker - INFO - Connection to scheduler broken. Closing without reporting. ID: Worker-25885526-3291-45fb-9c7d-275b006fdf1a Address tcp://127.0.0.1:62883 Status: Status.closing
2022-06-08 16:20:52,286 - distributed.scheduler - INFO - Scheduler closing...
2022-06-08 16:20:52,286 - distributed.scheduler - INFO - Scheduler closing all comms
Metadata
Metadata
Assignees
Labels
flaky testIntermittent failures on CI.Intermittent failures on CI.