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 dask_cudf.read_parquet regression for legacy timestamp data #15929

Merged
merged 7 commits into from
Jun 11, 2024
Merged
5 changes: 5 additions & 0 deletions python/dask_cudf/dask_cudf/io/parquet.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
from io import BufferedWriter, BytesIO, IOBase

import numpy as np
import pandas as pd
from pyarrow import dataset as pa_ds, parquet as pq

from dask import dataframe as dd
Expand Down Expand Up @@ -41,6 +42,10 @@ def _create_dd_meta(cls, dataset_info, **kwargs):
meta_pd = super()._create_dd_meta(dataset_info, **kwargs)

# Convert to cudf
# (drop unsupported timezone information)
for k, v in meta_pd.dtypes.items():
if isinstance(v, pd.DatetimeTZDtype) and v.tz is not None:
meta_pd[k] = meta_pd[k].dt.tz_localize(None)
meta_cudf = cudf.from_pandas(meta_pd)

# Re-set "object" dtypes to align with pa schema
Expand Down
9 changes: 4 additions & 5 deletions python/dask_cudf/dask_cudf/io/tests/test_parquet.py
Original file line number Diff line number Diff line change
Expand Up @@ -610,9 +610,8 @@ def test_timezone_column(tmpdir):
}
)
pdf.to_parquet(path)

# Check that `cudf` and `dask_cudf` results match
got = dask_cudf.read_parquet(path)
# cudf.read_parquet does not support reading timezone aware types yet
assert got["time"].dtype == pd.DatetimeTZDtype("ns", "UTC")
got["time"] = got["time"].astype("datetime64[ns]")
expected = cudf.read_parquet(path)
dd.assert_eq(got, expected)
expect = cudf.read_parquet(path)
dd.assert_eq(got, expect)
Loading