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

CategoricalAccessor.as_know() produces incorrect dtype with "p2p" shuffle enabled #11572

Open
rjzamora opened this issue Dec 2, 2024 · 0 comments
Labels
bug Something is broken

Comments

@rjzamora
Copy link
Member

rjzamora commented Dec 2, 2024

Context: dask/dask-expr#659 recently adjusted the default shuffle method used by series.unique() (from "tasks" to "p2p"). I don't think there was anything wrong with that particular PR. However, that change exposed the fact that .cat.as_known() does not properly preserve the original dtype of the categories when "p2p" is enabled:

import dask
import dask.dataframe as dd
from distributed import Client, LocalCluster

with LocalCluster() as cluster:
    with Client(cluster) as client:
        df = dd.from_dict({"qid": [1, 2, 1, 0, 2]}, npartitions=3)
    
        with dask.config.set({"dataframe.shuffle.method": "tasks"}):
            known_tasks = df.qid.astype("category").cat.as_known()
    
        with dask.config.set({"dataframe.shuffle.method": "p2p"}):
            known_p2p = df.qid.astype("category").cat.as_known()
    
        dd.assert_eq(known_tasks, known_p2p)
...
AssertionError: Attributes of Series are different

Attribute "dtype" are different
[left]:  CategoricalDtype(categories=[2, 0, 1], ordered=False, categories_dtype=int64)
[right]: CategoricalDtype(categories=['__UNKNOWN_CATEGORIES__', 2, 0, 1], ordered=False, categories_dtype=object)

I don't think the problem is in "p2p" itself. Rather, the proper metadata seems to be lost before the data is shuffled. For "tasks", the proper metadata is recovered after the intermediate compute step. However, we aren't as lucky for "p2p", because we round-trip the data to PyArrow.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something is broken
Projects
None yet
Development

No branches or pull requests

1 participant