Skip to content

Task stuck in "processing" on closed worker #6263

@bnaul

Description

@bnaul

Similar at a high level to #6198 but a slightly different manifestation: dashboard shows 9 remaining tasks (one is a parent task that spawned the other 8 by calling dd.read_parquet), but the Info page shows only the one parent task processing.
image

In the case of #6198 the worker showed up in the scheduler Info page (but would 404 when you tried to click through to its info); here the scheduler knows the workers are gone, but there are still tasks assigned to them anyway:

# Just the one parent task
In [33]: {k: v for k, v in c.processing().items() if v}
Out[33]: {'tcp://10.124.10.46:43097': ('partition_inputs-ea98882db0754e4497b1dcdd7d22e236',)}

# Here all the stragglers show up; each one is on a worker with status="closed"
In [34]: c.run_on_scheduler(lambda dask_scheduler: {ts.key: str(ts.processing_on) for ts in dask_scheduler.tasks.values() if ts.state in ("processing", "waiting")})
Out[34]:
{'partition_inputs-ea98882db0754e4497b1dcdd7d22e236': "<WorkerState 'tcp://10.124.10.46:43097', name: hardisty-2bab80b9-daskworkers-98d64ff7-28n8w, status: running, memory: 179, processing: 1>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 11771)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 11771)": "<WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closed, memory: 0, processing: 18>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 7365)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 7365)": "<WorkerState 'tcp://10.126.233.26:37341', name: 125, status: closed, memory: 0, processing: 23>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 3225)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 3225)": "<WorkerState 'tcp://10.126.167.29:36945', name: 232, status: closed, memory: 0, processing: 10>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 7711)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 7711)": "<WorkerState 'tcp://10.126.167.29:36945', name: 232, status: closed, memory: 0, processing: 10>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 4873)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 4873)": "<WorkerState 'tcp://10.127.22.29:43185', name: 412, status: closed, memory: 0, processing: 10>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 3331)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 3331)": "<WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closed, memory: 0, processing: 18>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 11393)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 11393)": "<WorkerState 'tcp://10.126.71.26:32909', name: 250, status: closed, memory: 0, processing: 10>",
 "('_split-ef14f4c130766e401c7f9804e1c5a7bd', 10716)": 'None',
 "('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 10716)": "<WorkerState 'tcp://10.126.71.26:32909', name: 250, status: closed, memory: 0, processing: 10>"}

Zooming in on the first closed worker 10.126.160.29:33011, relevant scheduler logs:

(.venv) ➜  model git:(master) ✗ kl hardisty-2bab80b9-daskscheduler-5c84ddcfd4-hfplp  | grep 10.126.160.29:33011
2022-05-03 02:05:00,218 - distributed.scheduler - INFO - Register worker <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: undefined, memory: 0, processing: 0>
2022-05-03 02:05:00,219 - distributed.scheduler - INFO - Starting worker compute stream, tcp://10.126.160.29:33011
2022-05-03 02:07:38,344 - distributed.scheduler - INFO - Retiring worker tcp://10.126.160.29:33011
2022-05-03 02:07:38,684 - distributed.active_memory_manager - INFO - Retiring worker tcp://10.126.160.29:33011; 21 keys are being moved away.
2022-05-03 02:07:39,572 - distributed.active_memory_manager - INFO - Retiring worker tcp://10.126.160.29:33011; 21 keys are being moved away.
2022-05-03 02:07:42,554 - distributed.active_memory_manager - INFO - Retiring worker tcp://10.126.160.29:33011; 10 keys are being moved away.
2022-05-03 02:07:43,888 - distributed.scheduler - INFO - Closing worker tcp://10.126.160.29:33011
2022-05-03 02:07:43,888 - distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 13, processing: 16>
2022-05-03 02:07:43,888 - distributed.core - INFO - Removing comms to tcp://10.126.160.29:33011
2022-05-03 02:07:43,895 - distributed.scheduler - INFO - Retired worker tcp://10.126.160.29:33011
2022-05-03 02:07:51,117 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.207.8:34749', name: 2314, status: running, memory: 11, processing: 5>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 12, processing: 0>, Key: ('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 7489)
2022-05-03 02:07:51,118 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.199.4:32777', name: 619, status: running, memory: 13, processing: 7>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 14, processing: 0>, Key: ('_split-ef14f4c130766e401c7f9804e1c5a7bd', 7535)
2022-05-03 02:07:51,119 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.7.24:33955', name: 1049, status: running, memory: 6, processing: 5>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 14, processing: 0>, Key: ('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 12280)
2022-05-03 02:07:51,121 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.113.32:41691', name: 572, status: running, memory: 8, processing: 5>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 15, processing: 0>, Key: ('_split-ef14f4c130766e401c7f9804e1c5a7bd', 2679)
2022-05-03 02:07:51,121 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.199.4:32777', name: 619, status: running, memory: 12, processing: 6>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 15, processing: 0>, Key: ('_split-ef14f4c130766e401c7f9804e1c5a7bd', 6989)
2022-05-03 02:07:51,121 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.133.31:42383', name: 2377, status: running, memory: 8, processing: 5>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 15, processing: 0>, Key: ('_split-ef14f4c130766e401c7f9804e1c5a7bd', 844)
2022-05-03 02:07:51,121 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.126.226.29:45983', name: 2094, status: running, memory: 7, processing: 6>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 15, processing: 0>, Key: ('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 10595)
2022-05-03 02:07:51,123 - distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tcp://10.127.195.7:42145', name: 2338, status: running, memory: 5, processing: 5>, Got: <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 16, processing: 0>, Key: ('_split-ef14f4c130766e401c7f9804e1c5a7bd', 11009)
2022-05-03 02:07:51,123 - distributed.scheduler - INFO - Register worker <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 16, processing: 0>
2022-05-03 02:07:51,128 - distributed.scheduler - INFO - Starting worker compute stream, tcp://10.126.160.29:33011
2022-05-03 02:07:55,963 - distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://10.126.160.29:33011', name: 77, status: closing_gracefully, memory: 5, processing: 0>
2022-05-03 02:07:55,963 - distributed.core - INFO - Removing comms to tcp://10.126.160.29:33011

And worker logs:

2022-05-03 02:04:58,377 - distributed.nanny - INFO -         Start Nanny at: 'tcp://10.126.160.29:45303'
2022-05-03 02:04:59,723 - distributed.worker - INFO -       Start worker at:  tcp://10.126.160.29:33011
2022-05-03 02:04:59,723 - distributed.worker - INFO -          Listening to:  tcp://10.126.160.29:33011
2022-05-03 02:04:59,723 - distributed.worker - INFO -          dashboard at:        10.126.160.29:39677
2022-05-03 02:04:59,723 - distributed.worker - INFO - Waiting to connect to: tcp://hardisty-2bab80b9-daskscheduler:8786
2022-05-03 02:04:59,723 - distributed.worker - INFO - -------------------------------------------------
2022-05-03 02:04:59,723 - distributed.worker - INFO -               Threads:                          4
2022-05-03 02:04:59,723 - distributed.worker - INFO -                Memory:                   7.82 GiB
2022-05-03 02:04:59,723 - distributed.worker - INFO -       Local Directory: /src/dask-worker-space/worker-7181cv4y
2022-05-03 02:04:59,723 - distributed.worker - INFO - -------------------------------------------------
2022-05-03 02:05:00,219 - distributed.worker - INFO -         Registered to: tcp://hardisty-2bab80b9-daskscheduler:8786
2022-05-03 02:05:00,220 - distributed.worker - INFO - -------------------------------------------------
2022-05-03 02:05:00,221 - distributed.core - INFO - Starting established connection
2022-05-03 02:07:02,854 - distributed.utils_perf - INFO - full garbage collection released 473.14 MiB from 28 reference cycles (threshold: 9.54 MiB)
2022-05-03 02:07:44,775 - distributed.worker - INFO - -------------------------------------------------
2022-05-03 02:07:46,212 - distributed.worker - INFO - Stopping worker at tcp://10.126.160.29:33011
2022-05-03 02:07:46,212 - distributed.worker - INFO - Not reporting worker closure to scheduler
2022-05-03 02:07:46,219 - distributed.worker - INFO - Connection to scheduler broken. Closing without reporting.  Status: Status.closing
2022-05-03 02:07:51,128 - distributed.worker - INFO -         Registered to: tcp://hardisty-2bab80b9-daskscheduler:8786
2022-05-03 02:07:51,129 - distributed.worker - INFO - -------------------------------------------------
2022-05-03 02:07:51,129 - distributed.core - INFO - Starting established connection
2022-05-03 02:07:52,639 - distributed.worker - ERROR - Exception during execution of task ('read-parquet-14e1bf051c27f27510e4873d5022c6a8', 3331).
Traceback (most recent call last):
  File "/usr/local/lib/python3.9/site-packages/distributed/worker.py", line 3677, in execute
    result = await self.loop.run_in_executor(
  File "/usr/local/lib/python3.9/site-packages/tornado/platform/asyncio.py", line 257, in run_in_executor
    return self.asyncio_loop.run_in_executor(executor, func, *args)
  File "/usr/local/lib/python3.9/asyncio/base_events.py", line 814, in run_in_executor
    executor.submit(func, *args), loop=self)
  File "/usr/local/lib/python3.9/site-packages/distributed/_concurrent_futures_thread.py", line 127, in submit
    raise RuntimeError("cannot schedule new futures after shutdown")
RuntimeError: cannot schedule new futures after shutdown
2022-05-03 02:07:53,622 - distributed.nanny - INFO - Worker closed
2022-05-03 02:07:53,623 - distributed.nanny - ERROR - Worker process died unexpectedly
2022-05-03 02:07:54,097 - distributed.nanny - INFO - Closing Nanny at 'tcp://10.126.160.29:45303'. Report closure to scheduler: None
2022-05-03 02:07:54,098 - distributed.dask_worker - INFO - End worker

Also finally got a successful cluster dump 🎉

cc @gjoseph92 @fjetter @crusaderky @mrocklin

Metadata

Metadata

Assignees

Labels

bugSomething is brokendeadlockThe cluster appears to not make any progress

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions