Specifically, I found that index.dt.floor(...) can result in invalid divisions if the divisions already align with the period passed to floor. For example:
import dask.dataframe as dd
import pandas as pd
date_index = pd.date_range(start="2022-02-22", freq="16h", periods=12)
df = pd.Series(range(12), index=date_index)
ddf = dd.from_pandas(df, npartitions=3)
res = ddf.index.dt.floor("D")
sol = df.index.floor("D")
dd.assert_eq(res, sol)
Results in
Traceback (most recent call last):
File "/home/jcristharif/Code/dask/test.py", line 11, in <module>
dd.assert_eq(res, sol)
File "/home/jcristharif/Code/dask/dask/dataframe/utils.py", line 537, in assert_eq
assert_divisions(a, scheduler=scheduler)
File "/home/jcristharif/Code/dask/dask/dataframe/utils.py", line 616, in assert_divisions
assert index(df).max() < ddf.divisions[i + 1]
AssertionError
This error indicates that the max of a partition isn't less than the upper bound for that partition as stated by the divisions. This is because the divisions were adjusted by map_partitions (with .floor("D") applied), and that adjustment is non-linear (multiple values all mapped to the same output). ceil is also broken for the same reason, and there may be other dt methods with the same issue.
Action items:
[ ] Special case ceil & floor methods in the DatetimeAccessor class to properly adjust the divisions given an Index with known divisions
[ ] Evaluate other dt methods for similar bugs, and fix those as well (or create specific issues for them as found)
[ ] Improve our test coverage for the DatetimeAccessor, covering both Series.dt and Index.dt (with known and unknown divisions).
Specifically, I found that
index.dt.floor(...)
can result in invalid divisions if the divisions already align with the period passed to floor. For example:Results in
This error indicates that the max of a partition isn't less than the upper bound for that partition as stated by the divisions. This is because the divisions were adjusted by
map_partitions
(with.floor("D")
applied), and that adjustment is non-linear (multiple values all mapped to the same output).ceil
is also broken for the same reason, and there may be otherdt
methods with the same issue.Action items:
ceil
&floor
methods in theDatetimeAccessor
class to properly adjust the divisions given anIndex
with known divisionsdt
methods for similar bugs, and fix those as well (or create specific issues for them as found)DatetimeAccessor
, covering bothSeries.dt
andIndex.dt
(with known and unknown divisions).