From adbbefd212430e5b96c1b1acfaacff45972a4478 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 9 Sep 2024 13:27:10 +0800 Subject: [PATCH] Support yyyyMMdd in GetTimestamp operator for LEGACY mode Signed-off-by: Chong Gao --- docs/compatibility.md | 3 +++ .../src/main/python/date_time_test.py | 11 ++++++++++- .../sql/rapids/datetimeExpressions.scala | 19 +++++++++++++++++-- 3 files changed, 30 insertions(+), 3 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index f9af6764498..6be6607b04f 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -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` @@ -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 diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 0c877f00238..653699dd98b 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -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 @@ -460,6 +460,15 @@ def test_to_timestamp(parser_policy): { "spark.sql.legacy.timeParserPolicy": parser_policy}) +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])") + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, gen).select(f.to_timestamp(f.col("a"), "yyyyMMdd")), + { + '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']) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 3169d6bc543..9653fc6be71 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -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 */ @@ -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 => + // 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