-
-
Notifications
You must be signed in to change notification settings - Fork 718
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
Fix scheduler transition error on memory->erred
#8549
Changes from 2 commits
069c283
095ebdd
de4caa9
d6678b6
7c13025
b8ccfa4
723bcb5
1254620
69e6c83
d6abfd1
fcd7f0d
677af84
47e6ea0
1efefa4
071ac28
68ea88e
5884bf5
9ee578a
5ee1b6a
d8939ad
6da7824
2375647
f6dfe39
e054a38
87c2b38
449f447
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1964,6 +1964,7 @@ | |
) | ||
|
||
v = a_recs.get(key, finish) | ||
# The inner rec has higher priority? Is that always desired? | ||
func = self._TRANSITIONS_TABLE["released", v] | ||
b_recs, b_cmsgs, b_wmsgs = func(self, key, stimulus_id) | ||
|
||
|
@@ -2082,7 +2083,11 @@ | |
assert not ts.who_has | ||
assert not ts.processing_on | ||
for dts in ts.dependencies: | ||
assert dts.state not in {"forgotten", "erred"} | ||
assert dts.state not in {"forgotten", "erred"}, ( | ||
ts, | ||
dts, | ||
hendrikmakait marked this conversation as resolved.
Show resolved
Hide resolved
|
||
self.transition_log, | ||
) | ||
|
||
if ts.has_lost_dependencies: | ||
return {key: "forgotten"}, {}, {} | ||
|
@@ -2480,7 +2485,9 @@ | |
recommendations[key] = "forgotten" | ||
elif ts.has_lost_dependencies: | ||
recommendations[key] = "forgotten" | ||
elif ts.who_wants or ts.waiters: | ||
elif (ts.who_wants or ts.waiters) and not any( | ||
dts.state in ("erred",) for dts in ts.dependencies | ||
hendrikmakait marked this conversation as resolved.
Show resolved
Hide resolved
|
||
): | ||
recommendations[key] = "waiting" | ||
|
||
for dts in ts.waiters or (): | ||
|
@@ -2505,14 +2512,13 @@ | |
assert ts.exception_blame | ||
assert not ts.who_has | ||
assert not ts.waiting_on | ||
assert not ts.waiters | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This assertion does not work in two-step transitions. |
||
|
||
failing_ts = ts.exception_blame | ||
assert failing_ts | ||
|
||
for dts in ts.dependents: | ||
dts.exception_blame = failing_ts | ||
if not dts.who_has: | ||
dts.exception_blame = failing_ts | ||
recommendations[dts.key] = "erred" | ||
|
||
report_msg = { | ||
|
@@ -2547,6 +2553,9 @@ | |
|
||
for dts in ts.dependents: | ||
if dts.state == "erred": | ||
# Does this make sense? | ||
# This goes via released | ||
# dts -> released -> waiting | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Agree this makes no sense to me either. Is there a unit test anywhere to shed light on it? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I haven't investigated this any further. |
||
recommendations[dts.key] = "waiting" | ||
|
||
w_msg = { | ||
|
@@ -2621,8 +2630,8 @@ | |
self, | ||
key: Key, | ||
stimulus_id: str, | ||
worker: str | None = None, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How can a task be processing without a worker? Is it when the worker it was processing on died and it caused the task to increase its suspicious count too much? If so, it may be a good idea to note it in a comment? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It appears that this has been superseded by other changes in this PR. I'll remove it and see if CI complains. |
||
*, | ||
worker: str, | ||
cause: Key | None = None, | ||
exception: Serialized | None = None, | ||
traceback: Serialized | None = None, | ||
|
@@ -2675,7 +2684,8 @@ | |
|
||
if not ts.erred_on: | ||
ts.erred_on = set() | ||
ts.erred_on.add(worker) | ||
if worker: | ||
ts.erred_on.add(worker) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the type in the function declaration should change to There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My bad, I didn't clean this up properly. |
||
if exception is not None: | ||
ts.exception = exception | ||
ts.exception_text = exception_text | ||
|
@@ -2699,8 +2709,9 @@ | |
) | ||
|
||
for dts in ts.dependents: | ||
dts.exception_blame = failing_ts | ||
recommendations[dts.key] = "erred" | ||
if dts.who_has: | ||
dts.exception_blame = failing_ts | ||
recommendations[dts.key] = "erred" | ||
|
||
for dts in ts.dependencies: | ||
if dts.waiters: | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't get this comment?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a general comment about the two-step transitions. The recommendations created by the first step are executed before the second step, which may create weird state (as it did in this case).