Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KeyError in Worker.handle_compute_task (causes deadlock) #5482

Closed
gjoseph92 opened this issue Oct 29, 2021 · 7 comments · Fixed by #5507 or #5525
Closed

KeyError in Worker.handle_compute_task (causes deadlock) #5482

gjoseph92 opened this issue Oct 29, 2021 · 7 comments · Fixed by #5507 or #5525

Comments

@gjoseph92
Copy link
Collaborator

Traceback (most recent call last):
  File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/worker.py", line 1237, in handle_scheduler
    await self.handle_stream(
  File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/core.py", line 564, in handle_stream
    handler(**merge(extra, msg))
  File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/worker.py", line 1937, in handle_compute_task
    self.tasks[key].nbytes = value
KeyError: "('slices-40d6794b777d639e9440f8f518224cfd', 2, 1)"
distributed.worker - INFO - Connection to scheduler broken.  Reconnecting...

I may be able to reproduce this if necessary. I was running a stackstac example notebook on binder against a Coiled cluster over wss, where the particular versions of things were causing a lot of errors (unrelated to dask). So I was frequently rerunning the same tasks, cancelling them, restarting the client, rerunning, etc. Perhaps this cancelling, restarting, rerunning is related?

@fjetter says

The only reason this keyerror can appear is if the compute instruction transitions its own dependency into a forgotten state which is very, very wrong.

Relevant code, ending at the line where the error occurs:

def handle_compute_task(
self,
*,
key,
function=None,
args=None,
kwargs=None,
task=no_value,
who_has=None,
nbytes=None,
priority=None,
duration=None,
resource_restrictions=None,
actor=False,
annotations=None,
stimulus_id=None,
):
self.log.append((key, "compute-task", stimulus_id, time()))
try:
ts = self.tasks[key]
logger.debug(
"Asked to compute an already known task %s",
{"task": ts, "stimulus_id": stimulus_id},
)
except KeyError:
self.tasks[key] = ts = TaskState(key)
ts.runspec = SerializedTask(function, args, kwargs, task)
if priority is not None:
priority = tuple(priority) + (self.generation,)
self.generation -= 1
if actor:
self.actors[ts.key] = None
ts.exception = None
ts.traceback = None
ts.exception_text = ""
ts.traceback_text = ""
ts.priority = priority
ts.duration = duration
if resource_restrictions:
ts.resource_restrictions = resource_restrictions
ts.annotations = annotations
recommendations = {}
scheduler_msgs = []
for dependency in who_has:
dep_ts = self.ensure_task_exists(
key=dependency,
stimulus_id=stimulus_id,
priority=priority,
)
# link up to child / parents
ts.dependencies.add(dep_ts)
dep_ts.dependents.add(ts)
if ts.state in READY | {"executing", "waiting", "resumed"}:
pass
elif ts.state == "memory":
recommendations[ts] = "memory"
scheduler_msgs.append(self.get_task_state_for_scheduler(ts))
elif ts.state in {
"released",
"fetch",
"flight",
"missing",
"cancelled",
"error",
}:
recommendations[ts] = "waiting"
else:
raise RuntimeError(f"Unexpected task state encountered {ts} {stimulus_id}")
for msg in scheduler_msgs:
self.batched_stream.send(msg)
self.transitions(recommendations, stimulus_id=stimulus_id)
# We received new info, that's great but not related to the compute-task
# instruction
self.update_who_has(who_has, stimulus_id=stimulus_id)
if nbytes is not None:
for key, value in nbytes.items():
self.tasks[key].nbytes = value

Scheduler code producing the message which causes this error:

def _task_to_msg(state: SchedulerState, ts: TaskState, duration: double = -1) -> dict:
"""Convert a single computational task to a message"""
ws: WorkerState
dts: TaskState
# FIXME: The duration attribute is not used on worker. We could safe ourselves the time to compute and submit this
if duration < 0:
duration = state.get_task_duration(ts)
msg: dict = {
"op": "compute-task",
"key": ts._key,
"priority": ts._priority,
"duration": duration,
"stimulus_id": f"compute-task-{time()}",
"who_has": {},
}
if ts._resource_restrictions:
msg["resource_restrictions"] = ts._resource_restrictions
if ts._actor:
msg["actor"] = True
deps: set = ts._dependencies
if deps:
msg["who_has"] = {
dts._key: [ws._address for ws in dts._who_has] for dts in deps
}
msg["nbytes"] = {dts._key: dts._nbytes for dts in deps}
if state._validate:
assert all(msg["who_has"].values())
task = ts._run_spec
if type(task) is dict:
msg.update(task)
else:
msg["task"] = task
if ts._annotations:
msg["annotations"] = ts._annotations
return msg

@fjetter
Copy link
Member

fjetter commented Nov 2, 2021

So I was frequently rerunning the same tasks, cancelling them, restarting the client, rerunning, etc. Perhaps this cancelling, restarting, rerunning is related?

Probably. What you're describing is basically what motivated #5046, see specifically #5046 (comment) and to some extend #4413 (comment) although the latter is a bit outdated. I'll need to update both (xref #5413)

The code should be much more robust to this situation but I guess I missed an edge case. Therefore, a transition log of the deadlocked worker would be very valuable.

@fjetter
Copy link
Member

fjetter commented Nov 2, 2021

@fjetter says

The only reason this keyerror can appear is if the compute instruction transitions its own dependency into a forgotten state which is very, very wrong.

Explanation below

  1. Schedule ensures that msg["who_has"].keys() == msg["nbytes"].keys()

msg["who_has"] = {
dts._key: [ws._address for ws in dts._who_has] for dts in deps
}
msg["nbytes"] = {dts._key: dts._nbytes for dts in deps}

  1. Worker ensures there exists a TaskState in Worker.tasks for every key in who_has

for dependency in who_has:
dep_ts = self.ensure_task_exists(
key=dependency,
stimulus_id=stimulus_id,
priority=priority,
)
# link up to child / parents
ts.dependencies.add(dep_ts)
dep_ts.dependents.add(ts)

  1. We perform a bunch of transitions. This is the only place where can actually drop another task again.

self.transitions(recommendations, stimulus_id=stimulus_id)

  1. This is where the KeyError is raised.

for key, value in nbytes.items():
self.tasks[key].nbytes = value

Since 1.) ensures nbytes.keys() == who_has.keys() and 2.) ensure there exists a task for all keys in who_has, therefore it ensures there is a task for every key in nbytes, 3. is the only way we could drop/forget a key since 4. is still in the same synchronous block.

The transition logic for that broken key could be either of

A) released -> waiting -> ready -> executing -> cancelled(executing)
B) released -> fetch -> flight -> cancelled(flight)

Above 2.) should then be a no-op 3.) should then trigger a

A) cancelled->fetch transitions to flight
B) cancelled->fetch transitions to resumed(flight). This one triggers a release_key and is as far as I can see the only source which could generate a forgotten recommendation.

I think this is some not well covered edge case in https://github.com/dask/distributed/blame/76495965cf8d3fb5f54bb4b8d20279ae402e0957/distributed/worker.py#L2965-L2966

As I said, a transition log would be helpful since it would remove some of the hypothetical of this argumentation. It should basically just show us the series of decisions leading up to the event.

@bennnym
Copy link

bennnym commented Nov 3, 2021

At the request of @gjoseph92 I am posting my transition logs as a result of a deadlock:

log.txt

@gjoseph92
Copy link
Collaborator Author

^ For background @bennnym ran into this same KeyError:

distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO - Run out-of-band function 'lambda'
distributed.worker - ERROR - "('split-shuffle-1-80cde6aa3e7938b28ad0dfe2387b063e', 5, (2, 3))"
Traceback (most recent call last):
  File "/opt/conda/envs/coiled/lib/python3.8/site-packages/distributed/worker.py", line 1237, in handle_scheduler
    await self.handle_stream(
  File "/opt/conda/envs/coiled/lib/python3.8/site-packages/distributed/core.py", line 564, in handle_stream
    handler(**merge(extra, msg))
  File "/opt/conda/envs/coiled/lib/python3.8/site-packages/distributed/worker.py", line 1937, in handle_compute_task
    self.tasks[key].nbytes = value
KeyError: "('split-shuffle-1-80cde6aa3e7938b28ad0dfe2387b063e', 5, (2, 3))"
distributed.worker - INFO - Connection to scheduler broken.  Reconnecting...

This error then caused a deadlock (#5480). We confirmed it was also due to #5481 by checking for queued messages in the worker's batched_stream:

>>> client.run(
...     lambda dask_worker: (
...         len(dask_worker.batched_stream.buffer),
...         dask_worker.batched_stream.please_stop
...     )
... )
{'tls://10.12.16.108:35519': (0, False),
 'tls://10.12.16.88:42647': (0, False),
 'tls://10.12.18.155:41501': (0, False),
 'tls://10.12.19.84:40143': (0, False),
 'tls://10.12.20.221:40545': (0, False),
 'tls://10.12.21.210:45285': (0, False),
 'tls://10.12.21.217:35583': (0, False),
 'tls://10.12.21.39:43355': (0, False),
 'tls://10.12.23.137:44833': (0, False),
 'tls://10.12.23.225:36857': (0, False),
 'tls://10.12.23.48:36147': (620, True),  <-- this one has enqueued messages and a stopped `_background_send` coroutine
 'tls://10.12.24.247:42369': (0, False),
 'tls://10.12.26.157:42411': (0, False),
 'tls://10.12.27.196:44821': (0, False),
 'tls://10.12.27.70:40899': (0, False),
 'tls://10.12.28.196:32955': (0, False),
 'tls://10.12.29.12:40313': (0, False),
 'tls://10.12.30.116:45129': (0, False),
 'tls://10.12.30.157:34201': (0, False),
 'tls://10.12.31.58:36025': (0, False)}

So we then got transition logs for the key in question, which Ben has attached above.

@bennnym I forgot, could you also post the output of client.get_versions?

@bennnym
Copy link

bennnym commented Nov 3, 2021

@gjoseph92 my code was running in a notebook and my kernell died, sorry. That is all I have

@gjoseph92
Copy link
Collaborator Author

Do you know offhand what versions of dask and distributed you were running? It may show in your Coiled software environment.

@bennnym
Copy link

bennnym commented Nov 3, 2021

I was just installing dask[distributed] so I assume the latest version

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants