From 8efa64ea61905969423bbfcc11353817c7cc1bca Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Tue, 11 Jun 2024 11:31:20 -0500 Subject: [PATCH] Fix `dask_cudf.read_parquet` regression for legacy timestamp data (#15929) cudf does not currently support timezone-aware datetime columns. For example: ```python pdf = pd.DataFrame( { "time": pd.to_datetime( ["1996-01-02", "1996-12-01"], utc=True, ), "x": [1, 2], } ) cudf.DataFrame.from_pandas(pdf) ``` ``` NotImplementedError: cuDF does not yet support timezone-aware datetimes ``` However, `cudf.read_parquet` **does** allow you to read this same data from a Parquet file. This PR adds a simple fix to allow the same data to be read with `dask_cudf`. The dask_cudf version was previously "broken" because it relies on upstream pyarrow logic to construct `meta` as a pandas DataFrame (and then we just convert `meta` from pandas to cudf). As illustrated in the example above, this direct conversion is not allowed when one or more columns contain timezone information. **Important Context** The actual motivation for this PR is to fix a **regression** in 24.06+ for older parquet files containing "legacy" timestamp types (e.g. `TIMESTAMP_MILLIS` and `TIMESTAMP_MICROS`). In `pyarrow 14.0.2` (used by cudf-24.04), these legacy types were not automatically translated to timezone-aware dtypes by pyarrow. In `pyarrow 16.1.0` (used by cudf-24.06+), the legacy types **ARE** automatically translated. Therefore, in moving from cudf-24.04 to cudf-24.06+, some `dask_cudf` users will find that they can no longer read the same parquet file containing legacy timestamp data. I'm not entirely sure if cudf should always allow users to read Parquet data with timezone-aware dtypes (e.g. if the timezone is **not** utc), but it definitely makes sense for cudf to ignore automatic/unnecessary timezone translations. Authors: - Richard (Rick) Zamora (https://github.com/rjzamora) Approvers: - Matthew Roeschke (https://github.com/mroeschke) - Lawrence Mitchell (https://github.com/wence-) URL: https://github.com/rapidsai/cudf/pull/15929 --- python/dask_cudf/dask_cudf/io/parquet.py | 5 +++++ python/dask_cudf/dask_cudf/io/tests/test_parquet.py | 9 ++++----- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/python/dask_cudf/dask_cudf/io/parquet.py b/python/dask_cudf/dask_cudf/io/parquet.py index fc962670c47..ba8b1e89721 100644 --- a/python/dask_cudf/dask_cudf/io/parquet.py +++ b/python/dask_cudf/dask_cudf/io/parquet.py @@ -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 @@ -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 diff --git a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py index f3e3911e6c7..620a917109e 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -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)