Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion dask_sql/mappings.py
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,8 @@ def sql_to_python_value(sql_type: str, literal_value: Any) -> Any:
dt = datetime.fromtimestamp(
int(literal_value.getTimeInMillis()) / 1000, timezone.utc
)

if sql_type == "DATE":
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would you mind adding a comment here linking to #296?

Suggested change
if sql_type == "DATE":
# address datetime format mismatch
# https://github.com/dask-contrib/dask-sql/issues/296
if sql_type == "DATE":

return dt.date()
return dt

elif sql_type.startswith("DECIMAL("):
Expand Down
10 changes: 6 additions & 4 deletions dask_sql/physical/rex/core/call.py
Original file line number Diff line number Diff line change
Expand Up @@ -196,14 +196,16 @@ def cast(self, operand, rex=None) -> SeriesOrScalar:
if not is_frame(operand):
return operand

output_type = str(rex.getType())
output_type = sql_to_python_type(output_type.upper())

return_column = cast_column_to_type(operand, output_type)
sql_output_type = str(rex.getType())
python_output_type = sql_to_python_type(sql_output_type.upper())
return_column = cast_column_to_type(operand, python_output_type)

if return_column is None:
return operand
else:
# handle datetime type specially
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above - just adding a link to #296

Suggested change
# handle datetime type specially
# address datetime format mismatch
# https://github.com/dask-contrib/dask-sql/issues/296

if sql_output_type == "DATE":
return_column = return_column.dt.date
return return_column


Expand Down
11 changes: 11 additions & 0 deletions tests/integration/test_filter.py
Original file line number Diff line number Diff line change
Expand Up @@ -68,3 +68,14 @@ def test_string_filter(c, string_table):
assert_frame_equal(
return_df, string_table.head(1),
)


def test_datetime_filter(c):
df = pd.DataFrame(
{"d_date": ["2001-08-01", "2001-08-02", "2001-08-03"], "val": [1, 2, 3]}
)
c.create_table("datetime_tbl1", df)
query = "SELECT val, d_date FROM datetime_tbl1 WHERE CAST(d_date as date) IN (date '2001-08-01', date '2001-08-03')"
result_df = c.sql(query).compute().reset_index(drop=True)
expected_df = pd.DataFrame({"val": [1, 3], "d_date": ["2001-08-01", "2001-08-03"]})
assert_frame_equal(result_df, expected_df, check_dtype=False)
10 changes: 10 additions & 0 deletions tests/integration/test_select.py
Original file line number Diff line number Diff line change
Expand Up @@ -115,3 +115,13 @@ def test_timezones(c, datetime_table):
result_df = result_df.compute()

assert_frame_equal(result_df, datetime_table)


def test_date_casting(c, datetime_table):
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinks we don't actually need datetime_table here? Since we only use the context here

Suggested change
def test_date_casting(c, datetime_table):
def test_date_casting(c):

# check date casting
query = "SELECT cast(timezone as date) as date1,cast(utc_timezone as date) as date2 FROM datetime_table "
result_df = c.sql(query).compute().astype(str)
expected_df = pd.DataFrame(
{"date1": ["2014-08-01"] * 3, "date2": ["2014-08-01"] * 3}
)
assert_frame_equal(result_df, expected_df, check_dtype=False)