-
-
Notifications
You must be signed in to change notification settings - Fork 27
Open
Description
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
Labels
No labels