Skip to content

cluster_dump roundtrips break non-stringified keys #8540

Open
@hendrikmakait

Description

@hendrikmakait

Describe the issue:

After we removed key stringification in #8083, keys can now be composite of multiple different types. Our cluster_dump functionality is not capable of correctly roundtripping those.

Minimal Complete Verifiable Example:

import dask.dataframe as dd
import pandas as pd
from distributed import Client
from distributed.cluster_dump import DumpArtefact

if __name__ == "__main__":
    c = Client()
    df = dd.from_pandas(pd.DataFrame({"a": [1, 2, 3]}), npartitions=3)
    result = df.compute()
    c.dump_cluster_state(filename="dump", format="yaml")
    dump = DumpArtefact.from_url("dump.yaml")
    dump.scheduler_story(df.__dask_keys__()[0])

raises

Traceback (most recent call last):
  File "/Users/hendrikmakait/projects/dask/distributed/foo.py", line 12, in <module>
    dump.scheduler_story(df.__dask_keys__()[0])
  File "/Users/hendrikmakait/projects/dask/distributed/distributed/cluster_dump.py", line 207, in scheduler_story
    for story in _scheduler_story(keys, log):
                 ^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/hendrikmakait/projects/dask/distributed/distributed/_stories.py", line 33, in scheduler_story
    if t[0] in keys_or_stimuli or keys_or_stimuli.intersection(t[3])
       ^^^^^^^^^^^^^^^^^^^^^^^
TypeError: unhashable type: 'list'

For msgpack, the error is the same.

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions