Skip to content

Categorical column turned to NaN after P2P Shuffle #8186

@NickSchouten

Description

@NickSchouten

Describe the issue:
If you P2P shuffle while your dataframe has a categorical column that column will be turned into all NaNs.

Minimal Complete Verifiable Example:

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

cluster = LocalCluster()
client = Client(cluster)
client.cluster.scale(1)


dask_df = dd.from_dict(
    {
        "b": ["a", "shuffle", "column"],
        "test": ["apple", "pear", "citrus"],
    },
    npartitions=2,
)
dask_df.test = dask_df.test.astype('category')  
print(list(dask_df.test.unique().compute())) # ['apple', 'pear', 'citrus']
dask_df = dask_df.shuffle(on="b").persist() 
print(list(dask_df.test.unique().compute())) # [nan]

Anything else we need to know?:
Probably due to the p2p shuffling.
dask_df = dask_df.shuffle(on="b", shuffle="tasks").persist()
does not have the issue.

If the column is not categorical it also does not happen.

Might be related to
#8183 and #8165

Environment:

Dask version: 2023.9.1
Python version: 3.10
Operating System: Linux
Install method (conda, pip, source): conda

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