Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support non-UTC timezone for casting from date type to timestamp type [databricks] #11462

Merged
merged 5 commits into from
Sep 23, 2024
Merged
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
2 changes: 0 additions & 2 deletions integration_tests/src/main/python/cast_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -726,8 +726,6 @@ def test_cast_int_to_string_not_UTC():
{"spark.sql.session.timeZone": "+08"})

not_utc_fallback_test_params = [(timestamp_gen, 'STRING'),
# python does not like year 0, and with time zones the default start date can become year 0 :(
(DateGen(start=date(1, 1, 1)), 'TIMESTAMP'),
(SetValuesGen(StringType(), ['2023-03-20 10:38:50', '2023-03-20 10:39:02']), 'TIMESTAMP')]

@allow_non_gpu('ProjectExec')
Expand Down
12 changes: 12 additions & 0 deletions integration_tests/src/main/python/date_time_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -684,3 +684,15 @@ def test_timestamp_millis_long_overflow():
def test_timestamp_micros(data_gen):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_micros(a)"))


@pytest.mark.skipif(not is_supported_time_zone(), reason="not all time zones are supported now, refer to https://github.com/NVIDIA/spark-rapids/issues/6839, please update after all time zones are supported")
@pytest.mark.parametrize('parser_policy', ['LEGACY', 'CORRECTED', 'EXCEPTION'], ids=idfn)
def test_date_to_timestamp(parser_policy):
assert_gpu_and_cpu_are_equal_sql(
lambda spark : unary_op_df(spark, date_gen),
"tab",
"SELECT cast(a as timestamp) from tab",
conf = {
"spark.sql.legacy.timeParserPolicy": parser_policy,
"spark.rapids.sql.incompatibleDateFormats.enabled": True})
Copy link
Collaborator

@firestarman firestarman Sep 18, 2024

Choose a reason for hiding this comment

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

Seems we only need spark.rapids.sql.incompatibleDateFormats.enabled being true for LEGACY policy?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Yes.

Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ abstract class CastExprMetaBase[INPUT <: UnaryExpression with TimeZoneAwareExpre
override def isTimeZoneSupported: Boolean = {
(fromType, toType) match {
case (TimestampType, DateType) => true // this is for to_date(...)
case (DateType, TimestampType) => true
case _ => false
}
}
Expand Down Expand Up @@ -631,6 +632,11 @@ object GpuCast {
zoneId.normalized())) {
shifted => shifted.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType))
}
case (DateType, TimestampType) if options.timeZoneId.isDefined =>
val zoneId = DateTimeUtils.getZoneId(options.timeZoneId.get)
withResource(input.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType))) { cv =>
GpuTimeZoneDB.fromTimestampToUtcTimestamp(cv, zoneId.normalized())
}
case _ =>
input.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType))
}
Expand Down
Loading