Skip to content

Commit

Permalink
Allow zoneinfo objects (#916)
Browse files Browse the repository at this point in the history
  • Loading branch information
mroeschke committed Apr 22, 2024
1 parent eec9e61 commit ec26733
Show file tree
Hide file tree
Showing 3 changed files with 47 additions and 24 deletions.
17 changes: 12 additions & 5 deletions fastparquet/test/test_api.py
Original file line number Diff line number Diff line change
Expand Up @@ -130,8 +130,14 @@ def test_sorted_row_group_columns(tempdir):
assert result == expected


@pytest.mark.xfail(
reason="Not supported by dask expressions",
raises=NotImplementedError,
strict=True,
)
def test_sorted_row_group_columns_with_filters(tempdir):
# fails up to 2021.08.1
dask = pytest.importorskip('dask')
dd = pytest.importorskip('dask.dataframe')
# create dummy dataframe
df = pd.DataFrame({'unique': [0, 0, 1, 1, 2, 2, 3, 3],
Expand All @@ -140,11 +146,12 @@ def test_sorted_row_group_columns_with_filters(tempdir):
'id1', 'id2',
'id1', 'id2']},
index=[0, 0, 1, 1, 2, 2, 3, 3])
df = dd.from_pandas(df, npartitions=2)
fn = os.path.join(tempdir, 'foo.parquet')
df.to_parquet(fn,
engine='fastparquet',
partition_on=['id'])
with dask.config.set({"dataframe.query-planning": False}):
df = dd.from_pandas(df, npartitions=2)
fn = os.path.join(tempdir, 'foo.parquet')
df.to_parquet(fn,
engine='fastparquet',
partition_on=['id'])
# load ParquetFile
pf = ParquetFile(fn)
filters = [('id', '==', 'id1')]
Expand Down
12 changes: 12 additions & 0 deletions fastparquet/test/test_converted_types.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
"""test_converted_types.py - tests for decoding data to their logical data types."""
import datetime
import os.path
import zoneinfo

import numpy as np
import pandas as pd
Expand Down Expand Up @@ -169,6 +170,17 @@ def test_tz_nonstring(tmpdir):
assert (event_df == round).all().all()


def test_tz_zoneinfo(tmpdir):
dti = pd.DatetimeIndex([pd.Timestamp(2020, 1, 1)], name="a").tz_localize(zoneinfo.ZoneInfo("UTC"))
df = pd.DataFrame({"a": dti})
fn = '{}/{}.parquet'.format(tmpdir, 'zoneinfo_tmp')
df.to_parquet(fn, compression='uncompressed', engine='fastparquet')
result = pd.read_parquet(fn, engine="fastparquet")
result_dtype = result.iloc[:, 0].dtype
assert isinstance(result_dtype, pd.DatetimeTZDtype)
assert str(result_dtype.tz) == "UTC"


def test_pandas_simple_type(tmpdir):
import pandas as pd
fn = os.path.join(tmpdir, "out.parquet")
Expand Down
42 changes: 23 additions & 19 deletions fastparquet/util.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
import operator
import re
import numbers
import zoneinfo

import numpy as np
import pandas as pd
Expand Down Expand Up @@ -417,25 +418,28 @@ def get_column_metadata(column, name, object_dtype=None):
'ordered': column.cat.ordered,
}
dtype = column.cat.codes.dtype
elif hasattr(dtype, 'tz'):
try:
stz = str(dtype.tz)
if "UTC" in stz and ":" in stz:
extra_metadata = {'timezone': stz.strip("UTC")}
elif len(str(stz)) == 3: # like "UTC", "CET", ...
extra_metadata = {'timezone': str(stz)}
elif getattr(dtype.tz, "zone", False):
extra_metadata = {'timezone': dtype.tz.zone}
elif "pytz" not in stz:
pd.Series([pd.to_datetime('now', utc=True)]).dt.tz_localize(stz)
extra_metadata = {'timezone': stz}
elif "Offset" in stz:
extra_metadata = {'timezone': f"{dtype.tz._minutes // 60:+03}:00"}
else:
raise KeyError
except Exception as e:
raise ValueError("Time-zone information could not be serialised: "
"%s, please use another" % str(dtype.tz)) from e
elif isinstance(dtype, pd.DatetimeTZDtype):
if isinstance(dtype.tz, zoneinfo.ZoneInfo):
extra_metadata = {'timezone': dtype.tz.key}
else:
try:
stz = str(dtype.tz)
if "UTC" in stz and ":" in stz:
extra_metadata = {'timezone': stz.strip("UTC")}
elif len(str(stz)) == 3: # like "UTC", "CET", ...
extra_metadata = {'timezone': str(stz)}
elif getattr(dtype.tz, "zone", False):
extra_metadata = {'timezone': dtype.tz.zone}
elif "pytz" not in stz:
pd.Series([pd.to_datetime('now', utc=True)]).dt.tz_localize(stz)
extra_metadata = {'timezone': stz}
elif "Offset" in stz:
extra_metadata = {'timezone': f"{dtype.tz._minutes // 60:+03}:00"}
else:
raise KeyError
except Exception as e:
raise ValueError("Time-zone information could not be serialised: "
"%s, please use another" % str(dtype.tz)) from e
else:
extra_metadata = None

Expand Down

0 comments on commit ec26733

Please sign in to comment.