-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Transition queued->memory causes AssertionError #7200
Comments
I somewhat intentionally didn't add a queued->memory transition since it seemed like a messed-up case to me, and I didn't want to cargo-cult it in just because other transitions of the same form were there. Seems it is in fact a messed-up case, but a possible one. So I guess we should just add the transition, which should be straightforward. Minimizing this test, on the other hand, might not be. |
I can work on minimising the test |
I'm working on this.
All implement the same logic to accept the task output from wherever it comes. However, the only case I can think of when this happens is when a cancellation request is already in flight but hasn't yet reached the worker, so the worker is going to lose the task immediately afterwards. I think they all have to be fixed to just disregard the transition. CC @fjetter |
This is not how stealing works. Stealing will be rejected if a key is already executing and nothing will be transitioned. I'm not sure how such a reschedule would actually occur. If it did we'd intentionally calculate a key on two workers simultaneously. |
I'm currently struggling to come up with a case where this transition would actually occur. I believe we eradicated these type of transitions when removing the worker reconnect. However, I'm not entirely sure. I do not consider it a valid API use to call
|
Won't it occur just by calling The implementation of
It seems like the implementation of
That specific case doesn't seem like something that should be a public API. I would guess that the only reason it works to call it in work stealing is that the situation is equivalent, because If I am leaning towards #7205 (comment) that |
So if I understand correctly, if I replace the contents of
with |
Yikes. story of test_stress_creation_and_deletion (breakpoint in [
("('sum-partial-0', 9, 7)", 'released', 'waiting', {}, 'update-graph-1666879902.9497185', 1666879902.9718037),
("('sum-0', 19, 14)", 'processing', 'memory', {"('sum-partial-0', 9, 7)": 'processing', "('random_sample-0', 19, 14)": 'released', "('random_sample-0', 14, 19)": 'released'}, 'task-finished-1666879904.9479938', 1666879904.956455)
("('sum-partial-0', 9, 7)", 'waiting', 'processing', {}, 'task-finished-1666879904.9479938', 1666879904.9565284
("('sum-0', 19, 14)", 'memory', 'released', {"('sum-0', 19, 14)": 'waiting', "('sum-partial-0', 9, 7)": 'waiting'}, 'handle-worker-cleanup-1666879904.976327', 1666879904.9796247)
("('sum-partial-0', 9, 7)", 'processing', 'released', {"('sum-partial-0', 9, 7)": 'waiting'}, 'handle-worker-cleanup-1666879904.976327', 1666879904.9796507)
("('sum-partial-0', 9, 7)", 'released', 'waiting', {"('sum-partial-0', 9, 7)": 'waiting', "('sum-0', 19, 14)": 'waiting'}, 'handle-worker-cleanup-1666879904.976327', 1666879904.979665)
("('sum-partial-0', 9, 7)", 'waiting', 'memory', <breakpoint>
] What I think I'm reading here:
|
My opinion: none of this is new. |
I think that's probably the best approach for now, but I find this concerning.
@crusaderky do you think we should also do this? It seems like the right thing to me. EDIT:
Either one of these is concerning.
One thing that does come to mind is that if an async function is a stream handler, the handler will be executed in the background: distributed/distributed/core.py Lines 835 to 842 in 0983731
So though we think of batched stream messages as being processed in order, async handlers are not necessarily processed in order, so it might be possible for part of an async handler to run after the batched stream has already closed. (xref #5443 a little.) The problem with this theory is that @crusaderky do you have a cluster dump of this case? Or at least a stimulus ID for the |
I can reproduce this locally as well. Pretty easily, actually.
Cluster dump is difficult because most of the workers shut down very quickly in this test. I can observe with a debugger, though.
stim ID in my case is just another task-finished- transition. What I could gather so far:
Still investigating for more. This test is currently xfailed. I think we should invest some time to make this work properly. |
Below the scheduler transition log.
the transition I think what happens is...
[
(
"FOO",
"released",
"waiting",
{},
"update-graph-1666955210.246825",
1666955210.291511,
),
(
"BAR",
"processing",
"memory",
{...},
"task-finished-1666955230.056821",
1666955230.1450088,
),
(
"FOO",
"waiting",
"processing",
{},
"task-finished-1666955230.056821",
1666955230.145199,
),
(
"BAR",
"memory",
"released",
{...},
"handle-worker-cleanup-WorkerA-1666955230.323402",
1666955230.327744,
),
(
"FOO",
"processing",
"released",
{...},
"handle-worker-cleanup-WorkerA-1666955230.323402",
1666955230.327806,
),
(
"FOO",
"released",
"waiting",
{...},
"handle-worker-cleanup-WorkerA-1666955230.323402",
1666955230.3278348,
),
(
"FOO",
"waiting",
"memory",
{...},
"task-finished-1666955230.232949",
1666955261.059616,
),
(
"FOO",
"memory",
"released",
{...},
"handle-worker-cleanup-WorkerA-1666955230.323402",
1666955262.808954,
),
(
"FOO",
"released",
"waiting",
{...},
"handle-worker-cleanup-WorkerA-1666955230.323402",
1666955262.809029,
),
] |
@fjette your analysis for waiting->memory is correct. I reproduced it deterministically in #7205. I cannot think of any way to reproduce no-worker->memory. f = c.submit(inc, 1, key="x", workers=["notexist"])
await asyncio.sleep(1)
await c.scatter({"x": 1}) but it won't cause the task to transition out of no-worker. |
I cannot, for the life of me, think of any legit use case for the other transitions. I've changed them to raise in the PR. |
This is resurfacing again in #7326 (comment) Reproducer @gen_cluster(client=True, nthreads=[("", 1)] * 3, timeout=5)
async def test_closed_worker_between_repeats(c, s, w1, w2, w3):
df = dask.datasets.timeseries(
start="2000-01-01",
end="2000-01-10",
dtypes={"x": float, "y": float},
freq="100 s",
seed=42,
)
out = dd.shuffle.shuffle(df, "x")
await c.compute(out.head(compute=False))
await w3.close()
await c.compute(out.tail(compute=False))
await w2.close()
await c.compute(out.head(compute=False)) cc @crusaderky Doesn't fail 100% deterministically but pretty reliably (about 1 out of 50) Story indicates that we're in the third compute and the scheduler receives a task-finished from Couldn't determine what's going on specifically, yet. |
While trying to reproduce #7063, I came across a different error, this one with queueing enabled.
The below reproducer is NOT minimal - there is likely quite a bit of simplification possible.
The test is green; however I read in the log:
What is happening:
This is timing-sensitive; if free-keys reached a before the task end, then steal_key would be cancelled and transition to forgotten without any messaging when it ends.
The text was updated successfully, but these errors were encountered: