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

Fix conversion from Categorical to pa.dictionary in read_parquet #10285

Merged
merged 5 commits into from
May 16, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion dask/dataframe/io/parquet/arrow.py
Original file line number Diff line number Diff line change
Expand Up @@ -1256,7 +1256,9 @@ def _create_dd_meta(cls, dataset_info):
# Make sure all categories are set to "unknown".
# Cannot include index names in the `cols` argument.
meta = clear_known_categories(
meta, cols=[c for c in categories if c not in meta.index.names]
meta,
cols=[c for c in categories if c not in meta.index.names],
dtype_backend=dtype_backend,
)

if partition_obj:
Expand Down
12 changes: 12 additions & 0 deletions dask/dataframe/io/tests/test_parquet.py
Original file line number Diff line number Diff line change
Expand Up @@ -4904,3 +4904,15 @@ def test_read_parquet_preserve_categorical_column_dtype(tmp_path):
index=[0, 0],
)
assert_eq(ddf, expected)


@PYARROW_MARK
@pytest.mark.skipif(not PANDAS_GT_150, reason="Requires pd.ArrowDtype")
def test_dtype_backend_categoricals(tmp_path):
df = pd.DataFrame({"a": pd.Series(["x", "y"], dtype="category"), "b": [1, 2]})
outdir = tmp_path / "out.parquet"
df.to_parquet(outdir, engine="pyarrow")
ddf = dd.read_parquet(outdir, engine="pyarrow", dtype_backend="pyarrow")
pdf = pd.read_parquet(outdir, engine="pyarrow", dtype_backend="pyarrow")
# Set sort_results=False because of pandas bug
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there an upstream issue for this?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My pr was already merged, will update tomorrow when nightlies are available

Copy link
Collaborator Author

@phofl phofl May 15, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice

assert_eq(ddf, pdf, sort_results=False)
10 changes: 9 additions & 1 deletion dask/dataframe/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ def strip_unknown_categories(x, just_drop_unknown=False):
return x


def clear_known_categories(x, cols=None, index=True):
def clear_known_categories(x, cols=None, index=True, dtype_backend=None):
"""Set categories to be unknown.

Parameters
Expand All @@ -273,7 +273,15 @@ def clear_known_categories(x, cols=None, index=True):
index : bool, optional
If True and x is a Series or DataFrame, set the clear known categories
in the index as well.
dtype_backend : string, optional
If set to PyArrow, the categorical dtype is implemented as a PyArrow
dictionary
"""
if dtype_backend == "pyarrow":
# Right now Categorical with PyArrow is implemented as dictionary and
# categorical accessor is not yet available
return x

if isinstance(x, (pd.Series, pd.DataFrame)):
x = x.copy()
if isinstance(x, pd.DataFrame):
Expand Down