Skip to content

Commit

Permalink
Support format 'yyyyMMdd HH:mm:ss' for legacy mode (#11658)
Browse files Browse the repository at this point in the history
Signed-off-by: Chong Gao <[email protected]>
  • Loading branch information
res-life authored Oct 28, 2024
1 parent e31a710 commit 91db040
Show file tree
Hide file tree
Showing 3 changed files with 24 additions and 5 deletions.
9 changes: 4 additions & 5 deletions docs/compatibility.md
Original file line number Diff line number Diff line change
Expand Up @@ -662,17 +662,16 @@ guaranteed to produce the same results as the CPU:
- `yyyymmdd`
- `yyyy/MM/dd HH:mm:ss`
- `yyyy-MM-dd HH:mm:ss`
- `yyyyMMdd HH:mm:ss`

LEGACY timeParserPolicy support has the following limitations when running on the GPU:

- 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. Only tested `UTC` and `Asia/Shanghai` timezones.
- When format is `yyyymmdd`, GPU only supports 8 digit strings. Spark supports like 7 digit
`2024101` string while GPU does not support. Only tested `UTC` and `Asia/Shanghai` timezones.

- When format is/contains `yyyyMMdd` or `yyyymmdd`, GPU only supports 8 digit strings for these formats.
Spark supports like 7 digit `2024101` string while GPU does not support. Only tested `UTC` and
`Asia/Shanghai` timezones.

## Formatting dates and timestamps as strings

Expand Down
18 changes: 18 additions & 0 deletions integration_tests/src/main/python/date_time_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -477,6 +477,24 @@ def test_formats_for_legacy_mode(format):
{'spark.sql.legacy.timeParserPolicy': 'LEGACY',
'spark.rapids.sql.incompatibleDateFormats.enabled': True})

# mm: minute; MM: month
@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.skipif(get_test_tz() != "Asia/Shanghai" and get_test_tz() != "UTC", reason="https://github.com/NVIDIA/spark-rapids/issues/11562")
def test_formats_for_legacy_mode_other_formats():
format = "yyyyMMdd HH:mm:ss"
# Test years after 1900,
gen = StringGen('(19[0-9]{2}|[2-9][0-9]{3})([0-9]{4}) [0-9]{2}:[0-9]{2}:[0-9]{2}')
assert_gpu_and_cpu_are_equal_sql(
lambda spark : unary_op_df(spark, gen),
"tab",
'''select unix_timestamp(a, '{}'),
from_unixtime(unix_timestamp(a, '{}'), '{}'),
date_format(to_timestamp(a, '{}'), '{}')
from tab
'''.format(format, format, format, format, format),
{'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")
@pytest.mark.parametrize("ansi_enabled", [True, False], ids=['ANSI_ON', 'ANSI_OFF'])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -641,6 +641,8 @@ object GpuToTimestamp {
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)"),
"yyyyMMdd HH:mm:ss" -> ParseFormatMeta(None, 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)"),
"yyyyMMdd" -> ParseFormatMeta(None, isTimestamp = false,
raw"\A\d{8}(\D|\s|\Z)"),
"yyyymmdd" -> ParseFormatMeta(None, isTimestamp = false,
Expand Down

0 comments on commit 91db040

Please sign in to comment.