Skip to content

Exception when executing task graph with HighLevelGraph after a shuffle #1129

@TomAugspurger

Description

@TomAugspurger

Describe the issue:

In geopandas/dask-geopandas#303, dask-geopandas has a report that its custom sjoin method fails with a TypeError under some conditions. Internally, that method constructs a HighLevelGraph.

The observed failure is a TypeError raised by geopandas because dask fails to substitute the concrete (geo)DataFrame for the key (name, partition_number) when executing the task graph.

I've managed to produce a dask / dask-expr only version:

Minimal Complete Verifiable Example:

import dask.dataframe as dd
import pandas as pd
import dask
import dask_expr

dask.config.set(scheduler="single-threaded")


l1 = dd.from_pandas(pd.DataFrame({"a": [1, 2], "b": [0, 0]}), npartitions=1).shuffle("a")
r1 = dd.from_pandas(pd.DataFrame({"a": [1, 3], "c": [1, 1]}), npartitions=1).shuffle("a")


def func(left, right):
    assert isinstance(left, pd.DataFrame), f"Wrong type {left}"
    return pd.merge(left, right, how="inner")


dsk = {}
name = "myjoin-" + dask.base.tokenize(l1, r1, ensure_deterministic=True)
for i in range(1):
    dsk[(name, i)] = (
        func,
        (l1._name, i),
        (r1._name, i),
    )

divisions = [None] * (len(dsk) + 1)
g1 = dask.highlevelgraph.HighLevelGraph.from_collections(name, dsk, dependencies=[l1, r1])
x1 = dask_expr.from_graph(g1, func(l1._meta, r1._meta), divisions, dsk.keys(), "myjoin")
x1.compute()

That fails with

---------------------------------------------------------------------------
AssertionError                            Traceback (most recent call last)
Cell In[3], line 30
     28 g1 = dask.highlevelgraph.HighLevelGraph.from_collections(name, dsk, dependencies=[l1, r1])
     29 x1 = dask_expr.from_graph(g1, func(l1._meta, r1._meta), divisions, dsk.keys(), "myjoin")
---> 30 x1.compute()

File [~/gh/geopandas/dask-geopandas/.direnv/python-3.10/lib/python3.10/site-packages/dask_expr/_collection.py:477](http://127.0.0.1:8889/~/gh/geopandas/dask-geopandas/.direnv/python-3.10/lib/python3.10/site-packages/dask_expr/_collection.py#line=476), in FrameBase.compute(self, fuse, **kwargs)
    475     out = out.repartition(npartitions=1)
    476 out = out.optimize(fuse=fuse)
--> 477 return DaskMethodsMixin.compute(out, **kwargs)

File [~/gh/geopandas/dask-geopandas/.direnv/python-3.10/lib/python3.10/site-packages/dask/base.py:376](http://127.0.0.1:8889/~/gh/geopandas/dask-geopandas/.direnv/python-3.10/lib/python3.10/site-packages/dask/base.py#line=375), in DaskMethodsMixin.compute(self, **kwargs)
    352 def compute(self, **kwargs):
    353     """Compute this dask collection
    354 
    355     This turns a lazy Dask collection into its in-memory equivalent.
   (...)
    374     dask.compute
    375     """
--> 376     (result,) = compute(self, traverse=False, **kwargs)
    377     return result

File [~/gh/geopandas/dask-geopandas/.direnv/python-3.10/lib/python3.10/site-packages/dask/base.py:662](http://127.0.0.1:8889/~/gh/geopandas/dask-geopandas/.direnv/python-3.10/lib/python3.10/site-packages/dask/base.py#line=661), in compute(traverse, optimize_graph, scheduler, get, *args, **kwargs)
    659     postcomputes.append(x.__dask_postcompute__())
    661 with shorten_traceback():
--> 662     results = schedule(dsk, keys, **kwargs)
    664 return repack([f(r, *a) for r, (f, a) in zip(results, postcomputes)])

Cell In[3], line 14, in func(left, right)
     13 def func(left, right):
---> 14     assert isinstance(left, pd.DataFrame), f"Wrong type {left}"
     15     return pd.merge(left, right, how="inner")

AssertionError: Wrong type ('rearrangebycolumn-8e5135a4ed46029cf9bb3c4bc7eff32e', 0)

As I mentioned, the shuffle there is important. Without that shuffle, things work fine.

Anything else we need to know?:

I'll take a look at this today.

Environment:

  • Dask version: '2024.8.1'
  • dask-expr version: '1.1.11'

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions