Skip to content

Commit

Permalink
Check for np.timedelta64 in as_timelike (#860)
Browse files Browse the repository at this point in the history
* check np.timedelta64 case

* add test

* Update dask_sql/physical/rex/core/call.py

Co-authored-by: Ayush Dattagupta <[email protected]>

Co-authored-by: Ayush Dattagupta <[email protected]>
  • Loading branch information
sarahyurick and ayushdg authored Oct 18, 2022
1 parent c19ea06 commit f9fbf5e
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 2 deletions.
2 changes: 1 addition & 1 deletion dask_sql/physical/rex/core/call.py
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ def as_timelike(op):
return np.timedelta64(op, "D")
elif isinstance(op, str):
return np.datetime64(op)
elif pd.api.types.is_datetime64_dtype(op):
elif pd.api.types.is_datetime64_dtype(op) or isinstance(op, np.timedelta64):
return op
else:
raise ValueError(f"Don't know how to make {type(op)} timelike")
Expand Down
19 changes: 18 additions & 1 deletion tests/integration/test_rex.py
Original file line number Diff line number Diff line change
Expand Up @@ -59,14 +59,31 @@ def test_intervals(c):
"""SELECT INTERVAL '3' DAY as "IN"
"""
)

expected_df = pd.DataFrame(
{
"IN": [pd.to_timedelta("3d")],
}
)
assert_eq(df, expected_df)

date1 = datetime(2021, 10, 3, 15, 53, 42, 47)
date2 = datetime(2021, 2, 28, 15, 53, 42, 47)
dates = dd.from_pandas(pd.DataFrame({"d": [date1, date2]}), npartitions=1)
c.register_dask_table(dates, "dates")
df = c.sql(
"""SELECT d + INTERVAL '5 days' AS "Plus_5_days" FROM dates
"""
)
expected_df = pd.DataFrame(
{
"Plus_5_days": [
datetime(2021, 10, 8, 15, 53, 42, 47),
datetime(2021, 3, 5, 15, 53, 42, 47),
]
}
)
assert_eq(df, expected_df)


def test_literals(c):
df = c.sql(
Expand Down

0 comments on commit f9fbf5e

Please sign in to comment.