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

Warn if tasks are submitted with identical keys but different run_spec #8185

Merged
merged 4 commits into from Feb 16, 2024

Conversation

fjetter
Copy link
Member

@fjetter fjetter commented Sep 14, 2023

This mitigates the impact of dask/dask#9888 by raising an exception when we detect that the submitted run_spec differs from the already known run_spec.

I believe to handle such a key collision gracefully we'd need to not only store the key itself but also a hash/token of the run_spec itself and pass this signature around everywhere. Alternatively, there was a proposal to use the TaskState object ID for this but the changes are the same.

A surprising amount of tests are failing with this change and I have to investigate more

distributed/scheduler.py Outdated Show resolved Hide resolved
@fjetter fjetter changed the title [WIP] Forbid the submission of tasks with differen run_spec Forbid the submission of tasks with differen run_spec Sep 14, 2023
@fjetter fjetter marked this pull request as ready for review September 14, 2023 15:06
@fjetter fjetter changed the title Forbid the submission of tasks with differen run_spec Fix possible deadlock when the same task key is used for different run_specs Sep 14, 2023
@fjetter fjetter added the deadlock The cluster appears to not make any progress label Sep 14, 2023
@fjetter fjetter changed the title Fix possible deadlock when the same task key is used for different run_specs Raise exception if tasks are submission with identical keys but different run_spec Sep 14, 2023
@fjetter fjetter changed the title Raise exception if tasks are submission with identical keys but different run_spec Raise exception if tasks are submitted with identical keys but different run_spec Sep 14, 2023
@github-actions
Copy link
Contributor

github-actions bot commented Sep 14, 2023

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

    27 files  ±  0      27 suites  ±0   10h 1m 29s ⏱️ - 9m 16s
 3 987 tests + 10   3 877 ✅ + 11    109 💤 ±0  1 ❌  - 1 
50 139 runs  +130  47 849 ✅ +132  2 289 💤  - 1  1 ❌  - 1 

For more details on these failures, see this check.

Results for commit 07974fd. ± Comparison against base commit 045dc64.

♻️ This comment has been updated with latest results.

Copy link
Collaborator

@crusaderky crusaderky left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm quite conflicted about this PR. I see a lot of pitfalls, and the only benefit would be to push the failure sooner in the process and make it more explicit.

I wonder if it wouldn't be simpler to just blindly retain the previous run_spec in case of resubmission?

Should we have a high-bandwidth brainstorming session about this?

distributed/scheduler.py Outdated Show resolved Hide resolved
elif (
# run_spec in the submitted graph may be None. This happens
# when an already persisted future is part of the graph
dsk.get(k) is not None
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it even possible that k is not in dsk? Short of a malformed graph

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As it turns out, there are plenty of possibilities. Most notably, this is the case when there are persisted futures as part of the graph

distributed/scheduler.py Outdated Show resolved Hide resolved
distributed/scheduler.py Outdated Show resolved Hide resolved
distributed/scheduler.py Outdated Show resolved Hide resolved
distributed/scheduler.py Outdated Show resolved Hide resolved
distributed/tests/test_client.py Outdated Show resolved Hide resolved
distributed/deploy/tests/test_adaptive.py Outdated Show resolved Hide resolved
@fjetter
Copy link
Member Author

fjetter commented Sep 15, 2023

I'm quite conflicted about this PR. I see a lot of pitfalls, and the only benefit would be to push the failure sooner in the process and make it more explicit.

I'm growing more convinced about this approach by the minute. Our assumptions that the key is uniquely identifying the entire task (not just the data) is so strongly baked into the system that we cannot afford to slip.
There are actually a couple of places in the code base where this is revealing a genuine bug. I share your concerns about tokenize as the comparison tool but atm this is merely an implementation detail and could be replaced with something a little smarter/more robust.

I will not be able to make time today for a high bandwidth session but I will also not be working on this today. If you are interested, I recommend checking out some of the remaining test failures. For instance, in P2P it appears that we have a genuine, potentially data-lossy bug in some situations because of this.

@fjetter fjetter marked this pull request as draft September 15, 2023 13:27
@crusaderky
Copy link
Collaborator

crusaderky commented Sep 18, 2023

Our assumptions that the key is uniquely identifying the entire task (not just the data) is so strongly baked into the system that we cannot afford to slip.

I agree that it's extremely unhealthy to let a task change its contents after it's been created.
What about my suggestion to just keep the oldest submission?

@fjetter
Copy link
Member Author

fjetter commented Sep 18, 2023

What about my suggestion to just keep the oldest submission?

Not too excited about this. We'd basically replace one inconsistency with another.

@crusaderky
Copy link
Collaborator

crusaderky commented Sep 18, 2023

What about my suggestion to just keep the oldest submission?

Not too excited about this. We'd basically replace one inconsistency with another.

Why? Assuming the user doesn't manually submit different tasks with the same hand-crafted key (if they want to shoot themselves in their feet, it's their problem), it would nicely work around the issue of the optimizer submitting different DAGs that produce the same output?

@fjetter
Copy link
Member Author

fjetter commented Sep 18, 2023

Why? Assuming the user doesn't manually submit different tasks with the same hand-crafted key (if they want to shoot themselves in their feet, it's their problem), it would nicely work around the issue of the optimizer submitting different DAGs that produce the same output?

I do not want to assume that there is only one possible way to achieve this corrupt state and guard against this.

When using the futures API, it is not that difficult to produce a state like this. Providing "hand crafted" keys is part of our API and users are using this (e.g. #3965 and #7551; unrelated but also problems because users are providing keys that break our assumptions). This is even the most natural way to scatter data.

I also have no idea how future optimizers will behave. I don't believe that blindly keeping the first known run_spec is the right way. I don't like guessing intent. Besides... the code right now actually does this (albeit buggy)!

@crusaderky
Copy link
Collaborator

crusaderky commented Feb 6, 2024

Status update

  • ✔️ Implementation done and polished
  • ✔️ Tests green
  • ✔️ replace the on-the-fly call to dask.config.set

This PR is blocked by

marks=pytest.mark.xfail(reason="https://github.com/dask/dask/issues/9888"),
),
],
)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@crusaderky
Copy link
Collaborator

Houston we have a problem.
CC @fjetter @hendrikmakait

The new check highlights behaviour in p2p shuffle where a piece of graph is eagerly sent up to the scheduler during graph definition time, then again with the same keys but different run_spec.

To my understanding this is a separate issue from dask/dask#9888 as it is doesn't go away with optimization.fuse.active=False.

You can easily see it in test_sort_values_with_existing_divisions, which in this PR i temporarily tweaked to be extra flaky. The test looks basic and harmless:

ddf = dd.from_pandas(df, npartitions=4)
with dask.config.set({"dataframe.shuffle.method": "p2p"}):
    ddf = ddf.set_index("a").sort_values("b")
    result = ddf.compute()

however, it trips the run_spec check, because

  1. it sends some keys to the scheduler up on set_index,
  2. waits for the result to return,
  3. then sends keys to the scheduler on sort_values,
  4. waits for the result to return,
  5. and finally sends the computation proper with compute.

Some keys are the same, but run_spec changes; this triggers a race condition where the scheduler didn't have the time to forget those keys yet.

Example exception:

E                           RuntimeError: Detected different `run_spec` for ('assign-4fae4863d4aa305fcb413e8ee9627427', 3) between two consecutive calls to `update_graph`. This is not allowed. Please ensure unique key names. If you are using a standard dask collections, consider releasing all the data before resubmitting another computation. More details and help can be found at https://github.com/dask/dask/issues/9888.
E                           
E                           Debugging information
E                           ---------------------
E                           old task: <TaskState ('assign-4fae4863d4aa305fcb413e8ee9627427', 3) released>
E                           old run_spec: (subgraph_callable-efbc1ec9d199a28827d79e2df96f380e, (('from_pandas-bcfd6d970dce28f6dcc856cd97ab2d6a', 3), '_partitions', ('set_partitions_pre-39118bf66e3cbd06e1bc4d2772ec676d', 3)), {})
E                           new run_spec: (subgraph_callable-387479db8a5e33870fa7493e21e0ad36, (('from_pandas-bcfd6d970dce28f6dcc856cd97ab2d6a', 3), '_partitions', 'getitem-b765ffd8b4899445a438edd4a4fe498a', 'a'), {})
E                           old token: ('tuple', [('SubgraphCallable', ('dict', [('tuple', ['assign-4fae4863d4aa305fcb413e8ee9627427', ('tuple', [(b'\x1f)\x90\xba\x17\xf5T\xad\x8dav\xf7\xba\xe3\x1bA\x13"Wt', []), '__dask_blockwise__0', '__dask_blockwise__1', '__dask_blockwise__2'])])]), 'assign-4fae4863d4aa305fcb413e8ee9627427', ('__dask_blockwise__0', '__dask_blockwise__1', '__dask_blockwise__2'), 'subgraph_callable-efbc1ec9d199a28827d79e2df96f380e'), ('tuple', [('tuple', ['from_pandas-bcfd6d970dce28f6dcc856cd97ab2d6a', 3]), '_partitions', ('tuple', ['set_partitions_pre-39118bf66e3cbd06e1bc4d2772ec676d', 3])]), ('dict', [])])
E                           new token: ('tuple', [('SubgraphCallable', ('dict', [('tuple', ['assign-4fae4863d4aa305fcb413e8ee9627427', 'getitem-set_partitions_pre-assign-4fae4863d4aa305fcb413e8ee9627427']), ('tuple', ['getitem-set_partitions_pre-assign-4fae4863d4aa305fcb413e8ee9627427', ('tuple', [(b'\x1f)\x90\xba\x17\xf5T\xad\x8dav\xf7\xba\xe3\x1bA\x13"Wt', []), '__dask_blockwise__0', '__dask_blockwise__1', ('tuple', [(b'~\x1fa\xba\x9f\x11Sk\n\xc4+\xc1\x14\xa2\xcbp\xab\xda\xf7\xc0', []), (b'G3!B^\xae\xa3\xae\x7f\xaek\xcb\xf4D\xcc\xf0\x00\x8a\x87M', []), ('list', [('tuple', [(b'6\xc7S\xf56\x17/2p\x91\xd2K\xdc/\x16\x06`\x1a)\xd0', []), '__dask_blockwise__0', '__dask_blockwise__3'])]), ('tuple', [(b'\xb6n\xd1\x897p\x92\x0b\xd5\xc7\xa6at\x10\xd7\x92\xcb\x92\xc3\xb0', []), ('list', [('list', ['_func', (b's_i\xa5\x8a\n4YL/j+<$\xc3\xd4\xbd\x18\x8a\xc3', [])]), ('list', ['_meta', [None, dtype('int64'), (b'\x7f\x0c\xeb\x89p\x07(\x8c!\xc7\x8d\x06\xd0{T\x15\xce:JZ', [b'\xda9\xa3\xee^kK\r2U\xbf\xef\x95`\x18\x90\xaf\xd8\x07\t']), [None, (b'\x7f\x0c\xeb\x89p\x07(\x8c!\xc7\x8d\x06\xd0{T\x15\xce:JZ', [b'\xda9\xa3\xee^kK\r2U\xbf\xef\x95`\x18\x90\xaf\xd8\x07\t'])]]]), ('list', ['divisions', [None, dtype('int64'), (b'\x9b\xd0\x9a\x85k\x01OtU\xe1\xb35X\x9c\x05\xcc\xe7e\x9f\x8d', [b'\xab/\xa5\n\xe2<\xe05\xba\xd2\xe7~\xc5\xe0\xbe\x05\xc2\xf4\xb8\x16']), [None, (b'\x9b\xd0\x9a\x85k\x01OtU\xe1\xb35X\x9c\x05\xcc\xe7e\x9f\x8d', [b'\xab/\xa5\n\xe2<\xe05\xba\xd2\xe7~\xc5\xe0\xbe\x05\xc2\xf4\xb8\x16'])]]])])])])])])]), 'assign-4fae4863d4aa305fcb413e8ee9627427', ('__dask_blockwise__0', '__dask_blockwise__1', '__dask_blockwise__2', '__dask_blockwise__3'), 'subgraph_callable-387479db8a5e33870fa7493e21e0ad36'), ('tuple', [('tuple', ['from_pandas-bcfd6d970dce28f6dcc856cd97ab2d6a', 3]), '_partitions', 'getitem-b765ffd8b4899445a438edd4a4fe498a', 'a']), ('dict', [])])
E                           old dependencies: {('from_pandas-bcfd6d970dce28f6dcc856cd97ab2d6a', 3), ('set_partitions_pre-39118bf66e3cbd06e1bc4d2772ec676d', 3)}
E                           new dependencies: {('from_pandas-bcfd6d970dce28f6dcc856cd97ab2d6a', 3)}

The use case won't fail in main because the key is always close to its released->forgotten transition when this happens (although in other cases it may be in memory; I'm not sure).

To me, the hard failure is very healthy behaviour, but at the same time this means we can't merge this PR in until after we fix shuffle to prevent the above.

crusaderky added a commit to crusaderky/distributed that referenced this pull request Feb 6, 2024
crusaderky added a commit to crusaderky/distributed that referenced this pull request Feb 6, 2024
@fjetter
Copy link
Member Author

fjetter commented Feb 7, 2024

Looking at the tokens you are posting, this still looks like dask/dask#9888. The config option you are referring to does not control blockwise fusion but rather low-level task fusing. Both are problematic but interestingly, the low level fusion is a little more robust.

See https://github.com/dask/dask/blob/7e04ff72cd598086babe13f181db2eb9d17ed402/dask/dataframe/optimize.py#L20-L33

This would also mean that this goes away with dask-expr (although I'm sure there are cases we can trigger a similar problem for arrays; one step at a time)

crusaderky added a commit to crusaderky/distributed that referenced this pull request Feb 15, 2024
crusaderky added a commit to crusaderky/distributed that referenced this pull request Feb 15, 2024
crusaderky added a commit to crusaderky/distributed that referenced this pull request Feb 15, 2024
crusaderky added a commit to fjetter/distributed that referenced this pull request Feb 15, 2024
crusaderky added a commit to fjetter/distributed that referenced this pull request Feb 15, 2024
@crusaderky
Copy link
Collaborator

Closes dask/dask#10905

tok_lhs != tok_rhs or deps_lhs != deps_rhs
) and ts.group not in tgs_with_bad_run_spec:
tgs_with_bad_run_spec.add(ts.group)
logger.warning(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Should we add a debug log regardless of ts.group not in tgs_with_bad_run_spec?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

distributed/tests/test_scheduler.py Outdated Show resolved Hide resolved
tok_lhs != tok_rhs or deps_lhs != deps_rhs
) and ts.group not in tgs_with_bad_run_spec:
tgs_with_bad_run_spec.add(ts.group)
logger.warning(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm wondering if it would be useful to propagate this warning to the user who submitted the graph instead of potentially burying it in the scheduler logs. Do you have thoughts on this?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

AFAIK we have no means to propagate warnings to the client? We'd need to build the infrastructure for it first

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wasn't entirely sure whether we had something that does this. Since that's not the case, forget what I said.

Comment on lines 4778 to 4779
tok_lhs: Any = tokenize(ts.run_spec, ensure_deterministic=True)
tok_rhs: Any = tokenize(dsk[k], ensure_deterministic=True)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we deal with the case where one of the two is deterministic but not the other?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@crusaderky
Copy link
Collaborator

@hendrikmakait all comments have been addressed

Copy link
Member

@hendrikmakait hendrikmakait left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, @crusaderky! LGTM, assuming CI is happy.

@hendrikmakait hendrikmakait merged commit d4380a7 into dask:main Feb 16, 2024
32 of 34 checks passed
@fjetter fjetter deleted the forbid_different_runspec branch February 23, 2024 17:50
crusaderky added a commit to crusaderky/distributed that referenced this pull request Feb 23, 2024
@DominikStiller
Copy link

Since version 2024.2.1, I am getting the "Detected different run_spec" error when using dask.array.linalg.svd_compressed (see attached log) with a distributed client. This happens both on dask arrays from Xarray DataArray's and on a randomly generated dask array. The issue does not occur every time.

Does this point to an issue in my code or with the dask.array.linalg code?

Log
2024-03-07 22:11:27,185 - distributed.scheduler - WARNING - Detected different `run_spec` for key ('getitem-769e519b145d702d5b6f3ccb12ecdb56-r1', 1, 0) between two consecutive calls to `update_graph`. This can cause failures and deadlocks down the line. Please ensure unique key names. If you are using a standard dask collections, consider releasing all the data before resubmitting another computation. More details and help can be found at https://github.com/dask/dask/issues/9888. 
Debugging information
---------------------
old task state: waiting
old run_spec: (<built-in function getitem>, (('qr-769e519b145d702d5b6f3ccb12ecdb56', 1, 0), 1), {})
new run_spec: (<function execute_task at 0x14dae93b8040>, (('qr-stack-getitem-769e519b145d702d5b6f3ccb12ecdb56-r1', 1, 0),), {})
old token: ('tuple', [('36c753f536172f327091d24bdc2f1606601a29d0', []), ('tuple', [('tuple', ['qr-769e519b145d702d5b6f3ccb12ecdb56', 1, 0]), 1]), ('dict', [])])
new token: ('tuple', [('913ceb5b5beb463a9010ec0790bc30002ca34164', []), ('tuple', [('tuple', ['qr-stack-getitem-769e519b145d702d5b6f3ccb12ecdb56-r1', 1, 0])]), ('dict', [])])
old dependencies: {('qr-769e519b145d702d5b6f3ccb12ecdb56', 1, 0)}
new dependencies: {('qr-stack-getitem-769e519b145d702d5b6f3ccb12ecdb56-r1', 1, 0)}

2024-03-07 22:11:27,215 - distributed.scheduler - WARNING - Detected different `run_spec` for key ('getitem-a7fb25e126d9fb7072604f61a677cec9-u2', 0, 0) between two consecutive calls to `update_graph`. This can cause failures and deadlocks down the line. Please ensure unique key names. If you are using a standard dask collections, consider releasing all the data before resubmitting another computation. More details and help can be found at https://github.com/dask/dask/issues/9888. 
Debugging information
---------------------
old task state: waiting
old run_spec: (<built-in function getitem>, (('svd-a7fb25e126d9fb7072604f61a677cec9-2', 0, 0), 0), {})
new run_spec: (<function execute_task at 0x14dae93b8040>, (('r-inner-svd-getitem-a7fb25e126d9fb7072604f61a677cec9-u2', 0, 0),), {})
old token: ('tuple', [('36c753f536172f327091d24bdc2f1606601a29d0', []), ('tuple', [('tuple', ['svd-a7fb25e126d9fb7072604f61a677cec9-2', 0, 0]), 0]), ('dict', [])])
new token: ('tuple', [('913ceb5b5beb463a9010ec0790bc30002ca34164', []), ('tuple', [('tuple', ['r-inner-svd-getitem-a7fb25e126d9fb7072604f61a677cec9-u2', 0, 0])]), ('dict', [])])
old dependencies: {('svd-a7fb25e126d9fb7072604f61a677cec9-2', 0, 0)}
new dependencies: {('r-inner-svd-getitem-a7fb25e126d9fb7072604f61a677cec9-u2', 0, 0)}

2024-03-07 22:11:27,561 - distributed.scheduler - WARNING - Detected different `run_spec` for key ('sum-sum-aggregate-qr-a7fb25e126d9fb7072604f61a677cec9', 3, 0) between two consecutive calls to `update_graph`. This can cause failures and deadlocks down the line. Please ensure unique key names. If you are using a standard dask collections, consider releasing all the data before resubmitting another computation. More details and help can be found at https://github.com/dask/dask/issues/9888. 
Debugging information
---------------------
old task state: waiting
old run_spec: (<function execute_task at 0x14dae93b8040>, ((subgraph_callable-82104eb30ade03273d3a1e81e03da052, (Compose(functools.partial(<function sum at 0x14dae9e59d30>, dtype=dtype('float64'), axis=(1,), keepdims=False), functools.partial(<function _concatenate2 at 0x14dace34cf40>, axes=[1])), [(subgraph_callable-e8a7c79807b1dc39611d1f609fa39cca, ('transpose-263070f06e10a9141fccf0b69de0efcc', 0, 0), ('mul-fb2964a69add9a29645401050021d9fa', 0, 3))])),), {})
new run_spec: (<function execute_task at 0x14dae93b8040>, ((subgraph_callable-82104eb30ade03273d3a1e81e03da052, (Compose(functools.partial(<function sum at 0x14dae9e59d30>, dtype=dtype('float64'), axis=(1,), keepdims=False), functools.partial(<function _concatenate2 at 0x14dace34cf40>, axes=[1])), [(subgraph_callable-94be14acf46fca928b596deb1288e971, ('mul-fb2964a69add9a29645401050021d9fa', 0, 3), ('getitem-1bb4125ac24aec5aa9a822d452df3245-q1', 0, 0), ('getitem-1bb4125ac24aec5aa9a822d452df3245-q2', 0, 0))])),), {})
old token: ('tuple', [('913ceb5b5beb463a9010ec0790bc30002ca34164', []), ('tuple', [('tuple', [('SubgraphCallable', ('dict', [('tuple', ['qr-a7fb25e126d9fb7072604f61a677cec9', 'transpose-qr-a7fb25e126d9fb7072604f61a677cec9']), ('tuple', ['transpose-qr-a7fb25e126d9fb7072604f61a677cec9', ('tuple', [('9c02d1d5538c77d8b71528257483854fa11a0da2', []), ('tuple', [('7e1f61ba9f11536b0ac42bc114a2cb70abdaf7c0', []), ('3fc8f5fe6a91c475721b0b78f509e5115bb2d857', []), ('list', ['__dask_blockwise__0']), ('tuple', [('b66ed1893770920bd5c7a6617410d792cb92c3b0', []), ('list', [('list', ['axes', ('tuple', [('0b56475c4c73a2ca74d7395c60b955b28aea804f', []), ('list', [1, 0])])])])])])])])]), 'qr-a7fb25e126d9fb7072604f61a677cec9', ('__dask_blockwise__0',), 'subgraph_callable-82104eb30ade03273d3a1e81e03da052'), ('tuple', [[[('6d7a2e9f4b1bb68b98aab7587a6acadcc9f01a99', []), ('tuple', []), ('dict', [('tuple', ['axes', ('list', [1])])])], [('54ba88fbd196d5c0ecae6e0e199d3c9f0379de7c', []), ('__seen', 23), ('dict', [('tuple', ['axis', ('tuple', [1])]), ('tuple', ['dtype', ('199f43faf9e8dede79d1ad17a76b597fdef78004', [])]), ('tuple', ['keepdims', False])])]], ('list', [('tuple', [('SubgraphCallable', ('dict', [('tuple', ['sum-59af6ccac08197d1fb38556ce20ca9c3', 'tensordot-sum-59af6ccac08197d1fb38556ce20ca9c3']), ('tuple', ['tensordot-sum-59af6ccac08197d1fb38556ce20ca9c3', ('tuple', [('__seen', 9), [('__seen', 28), ('__seen', 23), ('dict', [('tuple', ['dtype', ('__seen', 33)])])], ('list', [('tuple', [('__seen', 9), ('3de1fb6d8ffdaa7107380cb77b65976e152fd1d6', []), ('list', ['__dask_blockwise__0', '__dask_blockwise__1']), ('tuple', [('__seen', 13), ('list', [('list', ['axes', ('tuple', [('__seen', 17), ('list', [('tuple', [('__seen', 17), ('list', [1])]), ('tuple', [('__seen', 17), ('list', [0])])])])]), ('list', ['is_sparse', False])])])])]), ('tuple', [('__seen', 13), ('list', [('list', ['axis', ('tuple', [('__seen', 17), ('list', [1])])]), ('list', ['keepdims', True])])])])])]), 'sum-59af6ccac08197d1fb38556ce20ca9c3', ('__dask_blockwise__0', '__dask_blockwise__1'), 'subgraph_callable-e8a7c79807b1dc39611d1f609fa39cca'), ('tuple', ['transpose-263070f06e10a9141fccf0b69de0efcc', 0, 0]), ('tuple', ['mul-fb2964a69add9a29645401050021d9fa', 0, 3])])])])])]), ('dict', [])])
new token: ('tuple', [('913ceb5b5beb463a9010ec0790bc30002ca34164', []), ('tuple', [('tuple', [('SubgraphCallable', ('dict', [('tuple', ['qr-a7fb25e126d9fb7072604f61a677cec9', 'transpose-qr-a7fb25e126d9fb7072604f61a677cec9']), ('tuple', ['transpose-qr-a7fb25e126d9fb7072604f61a677cec9', ('tuple', [('9c02d1d5538c77d8b71528257483854fa11a0da2', []), ('tuple', [('7e1f61ba9f11536b0ac42bc114a2cb70abdaf7c0', []), ('3fc8f5fe6a91c475721b0b78f509e5115bb2d857', []), ('list', ['__dask_blockwise__0']), ('tuple', [('b66ed1893770920bd5c7a6617410d792cb92c3b0', []), ('list', [('list', ['axes', ('tuple', [('0b56475c4c73a2ca74d7395c60b955b28aea804f', []), ('list', [1, 0])])])])])])])])]), 'qr-a7fb25e126d9fb7072604f61a677cec9', ('__dask_blockwise__0',), 'subgraph_callable-82104eb30ade03273d3a1e81e03da052'), ('tuple', [[[('6d7a2e9f4b1bb68b98aab7587a6acadcc9f01a99', []), ('tuple', []), ('dict', [('tuple', ['axes', ('list', [1])])])], [('54ba88fbd196d5c0ecae6e0e199d3c9f0379de7c', []), ('__seen', 23), ('dict', [('tuple', ['axis', ('tuple', [1])]), ('tuple', ['dtype', ('199f43faf9e8dede79d1ad17a76b597fdef78004', [])]), ('tuple', ['keepdims', False])])]], ('list', [('tuple', [('SubgraphCallable', ('dict', [('tuple', ['dot-tensordot-transpose-sum-59af6ccac08197d1fb38556ce20ca9c3', ('tuple', [('__seen', 9), [('__seen', 28), ('__seen', 23), ('dict', [('tuple', ['dtype', ('__seen', 33)])])], ('list', [('tuple', [('__seen', 9), ('3de1fb6d8ffdaa7107380cb77b65976e152fd1d6', []), ('list', [('tuple', [('__seen', 9), ('__seen', 10), ('list', [('tuple', [('ef4e533c7c62179e376edc9bf90356741decb203', []), '__dask_blockwise__1', '__dask_blockwise__2'])]), ('tuple', [('__seen', 13), ('list', [('list', ['axes', ('tuple', [('__seen', 17), ('list', [1, 0])])])])])]), '__dask_blockwise__0']), ('tuple', [('__seen', 13), ('list', [('list', ['axes', ('tuple', [('__seen', 17), ('list', [('tuple', [('__seen', 17), ('list', [1])]), ('tuple', [('__seen', 17), ('list', [0])])])])]), ('list', ['is_sparse', False])])])])]), ('tuple', [('__seen', 13), ('list', [('list', ['axis', ('tuple', [('__seen', 17), ('list', [1])])]), ('list', ['keepdims', True])])])])]), ('tuple', ['sum-59af6ccac08197d1fb38556ce20ca9c3', 'dot-tensordot-transpose-sum-59af6ccac08197d1fb38556ce20ca9c3'])]), 'sum-59af6ccac08197d1fb38556ce20ca9c3', ('__dask_blockwise__0', '__dask_blockwise__1', '__dask_blockwise__2'), 'subgraph_callable-94be14acf46fca928b596deb1288e971'), ('tuple', ['mul-fb2964a69add9a29645401050021d9fa', 0, 3]), ('tuple', ['getitem-1bb4125ac24aec5aa9a822d452df3245-q1', 0, 0]), ('tuple', ['getitem-1bb4125ac24aec5aa9a822d452df3245-q2', 0, 0])])])])])]), ('dict', [])])
old dependencies: {('mul-fb2964a69add9a29645401050021d9fa', 0, 3), ('transpose-263070f06e10a9141fccf0b69de0efcc', 0, 0)}
new dependencies: {('getitem-1bb4125ac24aec5aa9a822d452df3245-q2', 0, 0), ('mul-fb2964a69add9a29645401050021d9fa', 0, 3), ('getitem-1bb4125ac24aec5aa9a822d452df3245-q1', 0, 0)}

2024-03-07 22:11:27,604 - distributed.scheduler - WARNING - Detected different `run_spec` for key ('sum-962ff5d42bfcf9dcb1caf353a65b5fad', 0, 18) between two consecutive calls to `update_graph`. This can cause failures and deadlocks down the line. Please ensure unique key names. If you are using a standard dask collections, consider releasing all the data before resubmitting another computation. More details and help can be found at https://github.com/dask/dask/issues/9888. 
Debugging information
---------------------
old task state: waiting
old run_spec: (<function execute_task at 0x14dae93b8040>, (('getitem-transpose-sum-962ff5d42bfcf9dcb1caf353a65b5fad', 0, 18),), {})
new run_spec: (subgraph_callable-93b65278d36411c5cf3ac2f2cf3d2428, (('getitem-36fceb35b8be7561758628514f65100c', 0, 18),), {})
old token: ('tuple', [('913ceb5b5beb463a9010ec0790bc30002ca34164', []), ('tuple', [('tuple', ['getitem-transpose-sum-962ff5d42bfcf9dcb1caf353a65b5fad', 0, 18])]), ('dict', [])])
new token: ('tuple', [('SubgraphCallable', ('dict', [('tuple', ['sum-962ff5d42bfcf9dcb1caf353a65b5fad', ('tuple', [('7e1f61ba9f11536b0ac42bc114a2cb70abdaf7c0', []), [('54ba88fbd196d5c0ecae6e0e199d3c9f0379de7c', []), ('tuple', []), ('dict', [('tuple', ['dtype', ('199f43faf9e8dede79d1ad17a76b597fdef78004', [])])])], ('list', ['__dask_blockwise__0']), ('tuple', [('b66ed1893770920bd5c7a6617410d792cb92c3b0', []), ('list', [('list', ['axis', ('tuple', [('0b56475c4c73a2ca74d7395c60b955b28aea804f', []), ('list', [1])])]), ('list', ['keepdims', True])])])])])]), 'sum-962ff5d42bfcf9dcb1caf353a65b5fad', ('__dask_blockwise__0',), 'subgraph_callable-93b65278d36411c5cf3ac2f2cf3d2428'), ('tuple', [('tuple', ['getitem-36fceb35b8be7561758628514f65100c', 0, 18])]), ('dict', [])])
old dependencies: {('getitem-transpose-sum-962ff5d42bfcf9dcb1caf353a65b5fad', 0, 18)}
new dependencies: {('getitem-36fceb35b8be7561758628514f65100c', 0, 18)}

@crusaderky
Copy link
Collaborator

crusaderky commented Mar 8, 2024

Since version 2024.2.1, I am getting the "Detected different run_spec" error when using dask.array.linalg.svd_compressed (see attached log) with a distributed client. This happens both on dask arrays from Xarray DataArray's and on a randomly generated dask array. The issue does not occur every time.

Not an error, a warning.
It highlights that your code is tripping dask/dask#9888.
It's a warning because it should be ok, to the best of our knowledge, at least as far as dask/dask#9888 is involved, but it's unhealthy behaviour that's fault-prone - particularly, it could be an indicator of key collision in case of manually crafted keys.

As a user, you could work around it by making sure you don't resubmit the same parts of your graph multiple times.
I expect your code looks something like this:

a = ... # dask collection
b = f(a)
a = a.persist()
b = b.persist()

the error will disappear if you change it to

a = ... # dask collection
a = a.persist()
b = f(a)
b = b.persist()

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
deadlock The cluster appears to not make any progress
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants