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 yyyyMMdd in GetTimestamp operator for LEGACY mode [databricks] #11449

Merged
merged 3 commits into from
Sep 23, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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: 3 additions & 0 deletions docs/compatibility.md
Original file line number Diff line number Diff line change
Expand Up @@ -651,6 +651,7 @@ guaranteed to produce the same results as the CPU:
- `dd/MM/yyyy`
- `yyyy/MM/dd`
- `yyyy-MM-dd`
- `yyyyMMdd`
- `yyyy/MM/dd HH:mm:ss`
- `yyyy-MM-dd HH:mm:ss`

Expand All @@ -659,6 +660,8 @@ LEGACY timeParserPolicy support has the following limitations when running on th
- Only 4 digit years are supported
- The proleptic Gregorian calendar is used instead of the hybrid Julian+Gregorian calendar
that Spark uses in legacy mode
- When format is `yyyyMMdd`, GPU only supports 8 digit strings. Spark supports like 7 digit
`2024101` string while GPU does not support.

## Formatting dates and timestamps as strings

Expand Down
12 changes: 11 additions & 1 deletion integration_tests/src/main/python/date_time_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
# limitations under the License.

import pytest
from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect, assert_gpu_and_cpu_error
from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect, assert_gpu_and_cpu_error, assert_gpu_and_cpu_are_equal_sql
from conftest import is_utc, is_supported_time_zone, get_test_tz
from data_gen import *
from datetime import date, datetime, timezone
Expand Down Expand Up @@ -459,6 +459,16 @@ def test_to_timestamp(parser_policy):
.select(f.col("a"), f.to_timestamp(f.col("a"), "yyyy-MM-dd HH:mm:ss")),
{ "spark.sql.legacy.timeParserPolicy": parser_policy})

@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")
def test_to_timestamp_legacy_mode_yyyyMMdd_format():
gen = StringGen("[0-9]{3}[1-9](0[1-9]|1[0-2])(0[1-9]|[1-2][0-9])")
firestarman marked this conversation as resolved.
Show resolved Hide resolved
assert_gpu_and_cpu_are_equal_sql(
lambda spark : unary_op_df(spark, gen),
"tab",
"select unix_timestamp(a, 'yyyyMMdd'), from_unixtime(unix_timestamp(a, 'yyyyMMdd'), 'yyyyMMdd'), date_format(to_timestamp(a, 'yyyyMMdd'), 'yyyyMMdd') from tab",
Copy link
Collaborator

Choose a reason for hiding this comment

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

Consider using multi-line strings

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Done

{
'spark.sql.legacy.timeParserPolicy': 'LEGACY',
'spark.rapids.sql.incompatibleDateFormats.enabled': True})

@tz_sensitive_test
@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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -640,7 +640,9 @@ object GpuToTimestamp {
"yyyy-MM-dd HH:mm:ss" -> ParseFormatMeta(Option('-'), isTimestamp = true,
raw"\A\d{4}-\d{1,2}-\d{1,2}[ T]\d{1,2}:\d{1,2}:\d{1,2}(\D|\s|\Z)"),
"yyyy/MM/dd HH:mm:ss" -> ParseFormatMeta(Option('/'), isTimestamp = true,
raw"\A\d{4}/\d{1,2}/\d{1,2}[ T]\d{1,2}:\d{1,2}:\d{1,2}(\D|\s|\Z)")
raw"\A\d{4}/\d{1,2}/\d{1,2}[ T]\d{1,2}:\d{1,2}:\d{1,2}(\D|\s|\Z)"),
"yyyyMMdd" -> ParseFormatMeta(None, isTimestamp = false,
raw"\A\d{8}(\D|\s|\Z)")
)

/** remove whitespace before month and day */
Expand Down Expand Up @@ -762,8 +764,21 @@ object GpuToTimestamp {
case RegexReplace(pattern, backref) =>
RegexReplace(pattern.replace('-', '/'), backref.replace('-', '/'))
}
case Some('-') | Some(_) | None =>
case Some('-') | Some(_) =>
regexReplaceRules
case None =>
firestarman marked this conversation as resolved.
Show resolved Hide resolved
// For formats like `yyyyMMdd` that do not contains separator,
// do not need to do regexp replacement rules
// Note: here introduced the following inconsistent behavior compared to Spark
// Spark's behavior:
// to_date('20240101', 'yyyyMMdd') = 2024-01-01
// to_date('202401 01', 'yyyyMMdd') = 2024-01-01
// to_date('2024 0101', 'yyyyMMdd') = null
// GPU behavior:
// to_date('20240101', 'yyyyMMdd') = 2024-01-01
// to_date('202401 01', 'yyyyMMdd') = null
// to_date('2024 0101', 'yyyyMMdd') = null
Seq()
}

// apply each rule in turn to the data
Expand Down
Loading