From e11023c752ca0c1dc03aabaf75cb71c92ec9933c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 30 May 2020 00:58:14 +0800 Subject: [PATCH 1/5] [SPARK-31868][SQL] Restore the behaviour week-based-year for 2.4 --- .../sql/catalyst/util/DateFormatter.scala | 2 +- .../util/DateTimeFormatterHelper.scala | 22 ++++++++---- .../catalyst/util/TimestampFormatter.scala | 2 +- .../expressions/DateExpressionsSuite.scala | 9 +++++ .../resources/sql-tests/inputs/datetime.sql | 6 ++++ .../sql-tests/results/ansi/datetime.sql.out | 34 ++++++++++++++++++- .../sql-tests/results/datetime-legacy.sql.out | 34 ++++++++++++++++++- .../sql-tests/results/datetime.sql.out | 34 ++++++++++++++++++- 8 files changed, 132 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 06e1cdc27e7d5..a71a04693286f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -56,7 +56,7 @@ class Iso8601DateFormatter( val specialDate = convertSpecialDate(s.trim, zoneId) specialDate.getOrElse { try { - val localDate = toLocalDate(formatter.parse(s)) + val localDate = toLocalDate(formatter.parse(s), locale) localDateToDays(localDate) } catch checkDiffResult(s, legacyFormatter.parse) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 353c074caa75e..2d271644a90ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.util import java.time._ import java.time.chrono.IsoChronology import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverStyle} -import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries} +import java.time.temporal._ import java.util.Locale import com.google.common.cache.CacheBuilder @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ trait DateTimeFormatterHelper { - private def getOrDefault(accessor: TemporalAccessor, field: ChronoField, default: Int): Int = { + private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = { if (accessor.isSupported(field)) { accessor.get(field) } else { @@ -39,15 +39,22 @@ trait DateTimeFormatterHelper { } } - protected def toLocalDate(accessor: TemporalAccessor): LocalDate = { + protected def toLocalDate(accessor: TemporalAccessor, locale: Locale): LocalDate = { val localDate = accessor.query(TemporalQueries.localDate()) // If all the date fields are specified, return the local date directly. if (localDate != null) return localDate + lazy val weekBasedYearField = WeekFields.of(locale).weekBasedYear() // Users may want to parse only a few datetime fields from a string and extract these fields // later, and we should provide default values for missing fields. // To be compatible with Spark 2.4, we pick 1970 as the default value of year. - val year = getOrDefault(accessor, ChronoField.YEAR, 1970) + val year = if (accessor.isSupported(ChronoField.YEAR)) { + accessor.get(ChronoField.YEAR) + } else if (accessor.isSupported(weekBasedYearField)) { + val year = accessor.get(weekBasedYearField) - 1 + return LocalDate.of(year, 12, 1).`with`(TemporalAdjusters.lastInMonth(DayOfWeek.SUNDAY)) + } else 1970 + val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1) val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1) LocalDate.of(year, month, day) @@ -74,8 +81,11 @@ trait DateTimeFormatterHelper { // Converts the parsed temporal object to ZonedDateTime. It sets time components to zeros // if they does not exist in the parsed object. - protected def toZonedDateTime(accessor: TemporalAccessor, zoneId: ZoneId): ZonedDateTime = { - val localDate = toLocalDate(accessor) + protected def toZonedDateTime( + accessor: TemporalAccessor, + zoneId: ZoneId, + locale: Locale): ZonedDateTime = { + val localDate = toLocalDate(accessor, locale) val localTime = toLocalTime(accessor) ZonedDateTime.of(localDate, localTime, zoneId) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 8428964d45707..8be179d1bda6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -79,7 +79,7 @@ class Iso8601TimestampFormatter( val parsed = formatter.parse(s) val parsedZoneId = parsed.query(TemporalQueries.zone()) val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId - val zonedDateTime = toZonedDateTime(parsed, timeZoneId) + val zonedDateTime = toZonedDateTime(parsed, timeZoneId, locale) val epochSeconds = zonedDateTime.toEpochSecond val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 1ca7380ead413..8bd280b08662e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1168,4 +1168,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkExceptionInExpression[ArithmeticException]( MillisToTimestamp(Literal(-92233720368547758L)), "long overflow") } + + test("SPARK-31868: Restore the behaviour week-based-year for 2.4") { + checkEvaluation( + new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, + Timestamp.valueOf("2017-12-31 13:33:33.0")) + checkEvaluation( + new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child, + Timestamp.valueOf("2017-12-31 00:00:00.0")) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 663c62f1a6f66..2f50b6c970a66 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -160,3 +160,9 @@ select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampF select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')); select from_csv('26/October/2015', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')); + +-- SPARK-31868: Restore the behaviour week-based-year for 2.4 +select to_timestamp('1969-01-01', 'YYYY-MM-dd'); +select to_timestamp('1969-12-31', 'YYYY-MM-dd'); +select to_timestamp('2018-01-01', 'YYYY-MM-dd'); +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss'); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 5857a0ac90c70..fca0cb6abb17c 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 116 +-- Number of queries: 120 -- !query @@ -999,3 +999,35 @@ struct<> -- !query output org.apache.spark.SparkUpgradeException You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select to_timestamp('1969-01-01', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +1968-12-29 00:00:00 + + +-- !query +select to_timestamp('1969-12-31', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +1968-12-29 00:00:00 + + +-- !query +select to_timestamp('2018-01-01', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +2017-12-31 00:00:00 + + +-- !query +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') +-- !query schema +struct +-- !query output +2017-12-31 13:33:33 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 8a726efafad89..939b33accb233 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 116 +-- Number of queries: 120 -- !query @@ -956,3 +956,35 @@ select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy struct> -- !query output {"date":2015-10-26} + + +-- !query +select to_timestamp('1969-01-01', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +1968-12-29 00:00:00 + + +-- !query +select to_timestamp('1969-12-31', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +1968-12-29 00:00:00 + + +-- !query +select to_timestamp('2018-01-01', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +2017-12-31 00:00:00 + + +-- !query +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') +-- !query schema +struct +-- !query output +2017-12-31 13:33:33 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 7cacaec42c813..f005eeeae8c96 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 116 +-- Number of queries: 120 -- !query @@ -971,3 +971,35 @@ struct<> -- !query output org.apache.spark.SparkUpgradeException You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + + +-- !query +select to_timestamp('1969-01-01', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +1968-12-29 00:00:00 + + +-- !query +select to_timestamp('1969-12-31', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +1968-12-29 00:00:00 + + +-- !query +select to_timestamp('2018-01-01', 'YYYY-MM-dd') +-- !query schema +struct +-- !query output +2017-12-31 00:00:00 + + +-- !query +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') +-- !query schema +struct +-- !query output +2017-12-31 13:33:33 From f2230b2a7f02f8f31f6a230582a95b7e67b4e726 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 30 May 2020 01:19:43 +0800 Subject: [PATCH 2/5] nit --- .../spark/sql/catalyst/util/DateTimeFormatterHelper.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 2d271644a90ec..c21abe2ca0889 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ trait DateTimeFormatterHelper { - private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = { + private def getOrDefault(accessor: TemporalAccessor, field: ChronoField, default: Int): Int = { if (accessor.isSupported(field)) { accessor.get(field) } else { From 0a6c730f1a61e2508e980a29c7ffbef77c711116 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 31 May 2020 13:51:58 +0800 Subject: [PATCH 3/5] temp --- .../util/DateTimeFormatterHelper.scala | 34 +++++++- .../catalyst/util/TimestampFormatter.scala | 3 +- .../expressions/DateExpressionsSuite.scala | 32 ++++++- .../resources/sql-tests/inputs/datetime.sql | 11 +++ .../sql-tests/results/ansi/datetime.sql.out | 83 +++++++++++++++++-- .../sql-tests/results/datetime-legacy.sql.out | 82 ++++++++++++++++-- .../sql-tests/results/datetime.sql.out | 83 +++++++++++++++++-- 7 files changed, 304 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index c21abe2ca0889..378fa65aeb4f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ trait DateTimeFormatterHelper { - private def getOrDefault(accessor: TemporalAccessor, field: ChronoField, default: Int): Int = { + private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = { if (accessor.isSupported(field)) { accessor.get(field) } else { @@ -44,20 +44,46 @@ trait DateTimeFormatterHelper { // If all the date fields are specified, return the local date directly. if (localDate != null) return localDate - lazy val weekBasedYearField = WeekFields.of(locale).weekBasedYear() + val weekFields = WeekFields.of(locale) + lazy val weekBasedYearField = weekFields.weekBasedYear() // Users may want to parse only a few datetime fields from a string and extract these fields // later, and we should provide default values for missing fields. // To be compatible with Spark 2.4, we pick 1970 as the default value of year. val year = if (accessor.isSupported(ChronoField.YEAR)) { accessor.get(ChronoField.YEAR) } else if (accessor.isSupported(weekBasedYearField)) { + // If we reach val year = accessor.get(weekBasedYearField) - 1 - return LocalDate.of(year, 12, 1).`with`(TemporalAdjusters.lastInMonth(DayOfWeek.SUNDAY)) + return LocalDate.of(year, 12, 1) + .`with`(TemporalAdjusters.lastInMonth(weekFields.getFirstDayOfWeek)) } else 1970 val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1) val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1) - LocalDate.of(year, month, day) + val weekBasedYear = if (accessor.isSupported(weekFields.weekBasedYear())) { + Option(accessor.get(weekFields.weekBasedYear())) + } else { + None + } + if (accessor.isSupported(weekFields.weekOfMonth())) { + Option(accessor.get(weekFields.weekOfMonth())) + } else { + None + } + val week = if (accessor.isSupported(weekFields.weekOfWeekBasedYear())) { + Option(accessor.get(weekFields.weekOfWeekBasedYear())) + } else { + None + } + val dayOfWeek = if (accessor.isSupported(weekFields.dayOfWeek())) { + Option(accessor.get(weekFields.dayOfWeek())) + } else { + None + } + + + val date = LocalDate.of(year, month, day) + date } private def toLocalTime(accessor: TemporalAccessor): LocalTime = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 8be179d1bda6b..6dab140832b81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -273,7 +273,8 @@ object LegacyDateFormats extends Enumeration { object TimestampFormatter { import LegacyDateFormats._ - val defaultLocale: Locale = Locale.US + val defaultLocale: Locale = + new Locale.Builder().setLocale(Locale.US).setExtension('u', "fw-mon").build() def defaultPattern(): String = s"${DateFormatter.defaultPattern} HH:mm:ss" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 8bd280b08662e..186190a605979 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1170,11 +1170,35 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-31868: Restore the behaviour week-based-year for 2.4") { +// checkEvaluation( +// new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, +// Timestamp.valueOf("2017-12-31 13:33:33.0")) +// checkEvaluation( +// new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child, +// Timestamp.valueOf("2017-12-31 00:00:00.0")) +// +// checkEvaluation( +// new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child, +// Timestamp.valueOf("1968-12-30 00:00:00.0")) +// +// checkEvaluation( +// new ParseToTimestamp(Literal("2018-46-7 13:33:33"), Literal("YYYY-ww-u HH:mm:ss")).child, +// Timestamp.valueOf("2018-11-17 13:33:33.0")) + +// checkEvaluation( +// new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, +// Timestamp.valueOf("2017-12-31 00:00:00.0")) + +// checkEvaluation( +// new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-W-dd")).child, +// Timestamp.valueOf("2017-12-31 00:00:00.0")) + checkEvaluation( - new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, - Timestamp.valueOf("2017-12-31 13:33:33.0")) - checkEvaluation( - new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child, + new ParseToTimestamp(Literal("1969 1 6 1"), Literal("yyyy M d u")).child, Timestamp.valueOf("2017-12-31 00:00:00.0")) + + checkEvaluation( + new ParseToTimestamp(Literal("1969 5 11 11"), Literal("YYYY M ww dd")).child, + Timestamp.valueOf("1969-01-01 00:00:00.0")) } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 2f50b6c970a66..198aac164c32e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -166,3 +166,14 @@ select to_timestamp('1969-01-01', 'YYYY-MM-dd'); select to_timestamp('1969-12-31', 'YYYY-MM-dd'); select to_timestamp('2018-01-01', 'YYYY-MM-dd'); select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss'); +select to_timestamp('1969 1 1', 'yyyy w u'); +select to_timestamp('1969 1 1', 'yyyy M u'); +select to_timestamp('1 1969 1', 'M YYYY u'); +select to_timestamp('1969 1 1', 'YYYY M u'); +select to_timestamp('1969 1 6 1', 'yyyy M d u'); +select to_timestamp('1969 1 5 1', 'yyyy M d u'); +-- YYYY-ww-dd +select to_timestamp('1969 11 11', 'YYYY ww dd'); +select to_timestamp('1969 2 11', 'YYYY W dd'); +select to_timestamp('1969 5 11 11', 'YYYY M ww dd'); + diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index fca0cb6abb17c..a1901cd0602f2 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 120 +-- Number of queries: 129 -- !query @@ -1006,7 +1006,7 @@ select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +1968-12-30 00:00:00 -- !query @@ -1014,7 +1014,7 @@ select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +1968-12-30 00:00:00 -- !query @@ -1022,7 +1022,7 @@ select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -2017-12-31 00:00:00 +2017-12-25 00:00:00 -- !query @@ -1030,4 +1030,77 @@ select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema struct -- !query output -2017-12-31 13:33:33 +2017-12-25 13:33:33 + + +-- !query +select to_timestamp('1969 1 1', 'yyyy w u') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 1 1', 'yyyy M u') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('1 1969 1', 'M YYYY u') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 1 1', 'YYYY M u') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 1 6 1', 'yyyy M d u') +-- !query schema +struct +-- !query output +1969-01-06 00:00:00 + + +-- !query +select to_timestamp('1969 1 5 1', 'yyyy M d u') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 11 11', 'YYYY ww dd') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 2 11', 'YYYY W dd') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 5 11 11', 'YYYY M ww dd') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 939b33accb233..85bb6a2eda7c5 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 120 +-- Number of queries: 129 -- !query @@ -963,7 +963,7 @@ select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +1968-12-30 00:00:00 -- !query @@ -971,7 +971,7 @@ select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +1968-12-30 00:00:00 -- !query @@ -979,7 +979,7 @@ select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -2017-12-31 00:00:00 +2018-01-01 00:00:00 -- !query @@ -987,4 +987,76 @@ select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema struct -- !query output -2017-12-31 13:33:33 +2018-01-01 13:33:33 + + +-- !query +select to_timestamp('1969 1 1', 'yyyy w u') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 1 1', 'yyyy M u') +-- !query schema +struct +-- !query output +1969-01-06 00:00:00 + + +-- !query +select to_timestamp('1 1969 1', 'M YYYY u') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 1 1', 'YYYY M u') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 1 6 1', 'yyyy M d u') +-- !query schema +struct +-- !query output +1969-01-06 00:00:00 + + +-- !query +select to_timestamp('1969 1 5 1', 'yyyy M d u') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 11 11', 'YYYY ww dd') +-- !query schema +struct +-- !query output +1969-03-10 00:00:00 + + +-- !query +select to_timestamp('1969 2 11', 'YYYY W dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 5 11 11', 'YYYY M ww dd') +-- !query schema +struct +-- !query output +1969-03-10 00:00:00 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index f005eeeae8c96..bcdd73d1c2197 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 120 +-- Number of queries: 129 -- !query @@ -978,7 +978,7 @@ select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +1968-12-30 00:00:00 -- !query @@ -986,7 +986,7 @@ select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +1968-12-30 00:00:00 -- !query @@ -994,7 +994,7 @@ select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -2017-12-31 00:00:00 +2017-12-25 00:00:00 -- !query @@ -1002,4 +1002,77 @@ select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema struct -- !query output -2017-12-31 13:33:33 +2017-12-25 13:33:33 + + +-- !query +select to_timestamp('1969 1 1', 'yyyy w u') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 1 1', 'yyyy M u') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('1 1969 1', 'M YYYY u') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 1 1', 'YYYY M u') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 1 6 1', 'yyyy M d u') +-- !query schema +struct +-- !query output +1969-01-06 00:00:00 + + +-- !query +select to_timestamp('1969 1 5 1', 'yyyy M d u') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1969 11 11', 'YYYY ww dd') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 2 11', 'YYYY W dd') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 + + +-- !query +select to_timestamp('1969 5 11 11', 'YYYY M ww dd') +-- !query schema +struct +-- !query output +1968-12-30 00:00:00 From bdab8bd86810c29b7e1c32f1c0bd022467cac1f5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 1 Jun 2020 15:58:16 +0800 Subject: [PATCH 4/5] update new approach --- .../util/DateTimeFormatterHelper.scala | 121 +++++++++++------- .../catalyst/util/TimestampFormatter.scala | 3 +- .../expressions/DateExpressionsSuite.scala | 78 +++++++---- .../resources/sql-tests/inputs/datetime.sql | 11 +- .../sql-tests/results/ansi/datetime.sql.out | 111 +++++++++++++--- .../sql-tests/results/datetime-legacy.sql.out | 90 +++++++++++-- .../sql-tests/results/datetime.sql.out | 111 +++++++++++++--- 7 files changed, 407 insertions(+), 118 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 378fa65aeb4f1..e89a11286a193 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -23,6 +23,8 @@ import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverSt import java.time.temporal._ import java.util.Locale +import scala.util.control.NonFatal + import com.google.common.cache.CacheBuilder import org.apache.spark.SparkUpgradeException @@ -31,59 +33,92 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ trait DateTimeFormatterHelper { - private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = { + private def getFieldValue( + accessor: TemporalAccessor, + field: TemporalField): Option[Int] = { if (accessor.isSupported(field)) { - accessor.get(field) + try { + Option(accessor.get(field)) + } catch { + case NonFatal(_) => None + } } else { - default + None } } + @throws[DateTimeException] protected def toLocalDate(accessor: TemporalAccessor, locale: Locale): LocalDate = { val localDate = accessor.query(TemporalQueries.localDate()) - // If all the date fields are specified, return the local date directly. + // If all the date fields are resolved(yMd or Ywu), return the local date directly. if (localDate != null) return localDate + var res = LocalDate.of(1970, 1, 1) + val weekFields = WeekFields.of(locale) - lazy val weekBasedYearField = weekFields.weekBasedYear() - // Users may want to parse only a few datetime fields from a string and extract these fields - // later, and we should provide default values for missing fields. - // To be compatible with Spark 2.4, we pick 1970 as the default value of year. - val year = if (accessor.isSupported(ChronoField.YEAR)) { - accessor.get(ChronoField.YEAR) - } else if (accessor.isSupported(weekBasedYearField)) { - // If we reach - val year = accessor.get(weekBasedYearField) - 1 - return LocalDate.of(year, 12, 1) - .`with`(TemporalAdjusters.lastInMonth(weekFields.getFirstDayOfWeek)) - } else 1970 - - val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1) - val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1) - val weekBasedYear = if (accessor.isSupported(weekFields.weekBasedYear())) { - Option(accessor.get(weekFields.weekBasedYear())) - } else { - None - } - if (accessor.isSupported(weekFields.weekOfMonth())) { - Option(accessor.get(weekFields.weekOfMonth())) - } else { - None - } - val week = if (accessor.isSupported(weekFields.weekOfWeekBasedYear())) { - Option(accessor.get(weekFields.weekOfWeekBasedYear())) - } else { - None + val weekBasedYearField = weekFields.weekBasedYear + var weekBasedYearEnabled = false + + val year = getFieldValue(accessor, weekBasedYearField).map { y => + weekBasedYearEnabled = true + y + }.orElse { + getFieldValue(accessor, ChronoField.YEAR) } - val dayOfWeek = if (accessor.isSupported(weekFields.dayOfWeek())) { - Option(accessor.get(weekFields.dayOfWeek())) + + val week = getFieldValue(accessor, weekFields.weekOfWeekBasedYear()) + val dayOfWeek = getFieldValue(accessor, weekFields.dayOfWeek()) + + // TODO: How to check 'W' week-of-month field, not like other week-based field, it always throw + // UnsupportedTemporalTypeException to get it even `accessor.isSupported` passed. + + if (weekBasedYearEnabled) { + // If the week-based-year field exists, only the week-based fields matters. + res.`with`(weekFields.weekOfWeekBasedYear, week.getOrElse(1).toLong) + .`with`(weekFields.dayOfWeek(), dayOfWeek.getOrElse(1).toLong) + .`with`(weekBasedYearField, year.get) } else { - None - } + if (year.isDefined) { + res = res.withYear(year.get) + } + val month = getFieldValue(accessor, ChronoField.MONTH_OF_YEAR) + val day = getFieldValue(accessor, ChronoField.DAY_OF_MONTH) - val date = LocalDate.of(year, month, day) - date + if (month.isDefined && week.isDefined) { + // check the week fall into the correct month of the year + res = res.`with`(weekFields.weekOfWeekBasedYear(), week.get) + if (res.getMonthValue != month.get) { + throw new DateTimeException( + s"week-of-week-based-year value: ${week.get} conflicts with month-of-year value:" + + s" ${month.get} which should be ${res.getMonthValue} instead.") + } + } else if (month.isDefined) { + res = res.withMonth(month.get) + } else if (week.isDefined) { + if (day.isDefined) { + throw new DateTimeException( + s"Can not use week-of-week-based-year and day-of-month together in non-week-based" + + s" mode.") + } + res = res.`with`(weekFields.weekOfWeekBasedYear, week.get) + } + + if (dayOfWeek.isDefined && day.isDefined) { + // check whether the days matches + res = res.`with`(weekFields.dayOfWeek(), dayOfWeek.get) + if (res.getDayOfMonth != day.get) { + throw new DateTimeException( + s"day-of-week value: ${dayOfWeek.get} conflicts with day-of-month value:" + + s" ${day.get} which should be ${res.getDayOfMonth} instead.") + } + } else if (day.isDefined) { + res = res.withDayOfMonth(day.get) + } else if (dayOfWeek.isDefined) { + res = res.`with`(weekFields.dayOfWeek(), dayOfWeek.get) + } + res + } } private def toLocalTime(accessor: TemporalAccessor): LocalTime = { @@ -99,10 +134,10 @@ trait DateTimeFormatterHelper { } else { 0 } - val minute = getOrDefault(accessor, ChronoField.MINUTE_OF_HOUR, 0) - val second = getOrDefault(accessor, ChronoField.SECOND_OF_MINUTE, 0) - val nanoSecond = getOrDefault(accessor, ChronoField.NANO_OF_SECOND, 0) - LocalTime.of(hour, minute, second, nanoSecond) + val minute = getFieldValue(accessor, ChronoField.MINUTE_OF_HOUR) + val second = getFieldValue(accessor, ChronoField.SECOND_OF_MINUTE) + val nanoSecond = getFieldValue(accessor, ChronoField.NANO_OF_SECOND) + LocalTime.of(hour, minute.getOrElse(0), second.getOrElse(0), nanoSecond.getOrElse(0)) } // Converts the parsed temporal object to ZonedDateTime. It sets time components to zeros diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 6dab140832b81..8be179d1bda6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -273,8 +273,7 @@ object LegacyDateFormats extends Enumeration { object TimestampFormatter { import LegacyDateFormats._ - val defaultLocale: Locale = - new Locale.Builder().setLocale(Locale.US).setExtension('u', "fw-mon").build() + val defaultLocale: Locale = Locale.US def defaultPattern(): String = s"${DateFormatter.defaultPattern} HH:mm:ss" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 186190a605979..728e349f28bff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1170,35 +1170,63 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-31868: Restore the behaviour week-based-year for 2.4") { -// checkEvaluation( -// new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, -// Timestamp.valueOf("2017-12-31 13:33:33.0")) -// checkEvaluation( -// new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child, -// Timestamp.valueOf("2017-12-31 00:00:00.0")) -// -// checkEvaluation( -// new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child, -// Timestamp.valueOf("1968-12-30 00:00:00.0")) -// -// checkEvaluation( -// new ParseToTimestamp(Literal("2018-46-7 13:33:33"), Literal("YYYY-ww-u HH:mm:ss")).child, -// Timestamp.valueOf("2018-11-17 13:33:33.0")) - -// checkEvaluation( -// new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, -// Timestamp.valueOf("2017-12-31 00:00:00.0")) - -// checkEvaluation( -// new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-W-dd")).child, -// Timestamp.valueOf("2017-12-31 00:00:00.0")) - + // TODO: Locale.US is Sunday started day of week, which affects the new formatter, + // while in 2.4 it's Monday first. checkEvaluation( - new ParseToTimestamp(Literal("1969 1 6 1"), Literal("yyyy M d u")).child, + new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child, Timestamp.valueOf("2017-12-31 00:00:00.0")) + checkEvaluation( + new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, + Timestamp.valueOf("2017-12-31 13:33:33.0")) + + checkEvaluation( + new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child, + Timestamp.valueOf("1968-12-30 00:00:00.0")) + + // the existence of 'W' is not for generating the timestamp but likely for checking whether + // the timestamp falling into it. + checkEvaluation( + new ParseToTimestamp(Literal("1969 5 1 2"), Literal("YYYY W w u")).child, + Timestamp.valueOf("1968-12-30 00:00:00.0")) + + checkEvaluation( + new ParseToTimestamp(Literal("1969 5 2"), Literal("YYYY W u")).child, + Timestamp.valueOf("1968-12-30 00:00:00.0")) + +// // the legacy parser does not support 'W' and results null, so SparkUpgradeException will come +// checkExceptionInExpression[SparkUpgradeException]( +// new ParseToTimestamp(Literal("1969 4 2"), Literal("YYYY W u")).child, "3.0") +// checkExceptionInExpression[SparkUpgradeException]( +// new ParseToTimestamp(Literal("5"), Literal("W")).child, "3.0") + + // https://bugs.openjdk.java.net/browse/JDK-8145633 + // Adjacent value parsing not supported for Localized Patterns + checkExceptionInExpression[SparkUpgradeException]( + new ParseToTimestamp(Literal("196940"), Literal("YYYYww")).child, "3.0") + + checkEvaluation( + new ParseToTimestamp(Literal("2020 1 3 2"), Literal("yyyy M w u")).child, + Timestamp.valueOf("2020-01-13 00:00:00.0")) + + checkEvaluation( + new ParseToTimestamp(Literal("2018-46-7 13:33:33"), Literal("YYYY-ww-u HH:mm:ss")).child, + Timestamp.valueOf("2018-11-17 13:33:33.0")) + + checkEvaluation( + new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, + Timestamp.valueOf("2018-03-11 00:00:00.0")) + + checkEvaluation( + new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-dd")).child, + null) + + // problem of first day of week change + checkExceptionInExpression[SparkUpgradeException]( + new ParseToTimestamp(Literal("1969 1 6 1"), Literal("yyyy M d u")).child, "3.0") + checkEvaluation( new ParseToTimestamp(Literal("1969 5 11 11"), Literal("YYYY M ww dd")).child, - Timestamp.valueOf("1969-01-01 00:00:00.0")) + Timestamp.valueOf("1969-03-09 00:00:00.0")) } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 198aac164c32e..199e9f8542ad4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -162,6 +162,7 @@ select from_csv('26/October/2015', 'time Timestamp', map('timestampFormat', 'dd/ select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy')); -- SPARK-31868: Restore the behaviour week-based-year for 2.4 +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss'); -- only the week-based year matters for the date part select to_timestamp('1969-01-01', 'YYYY-MM-dd'); select to_timestamp('1969-12-31', 'YYYY-MM-dd'); select to_timestamp('2018-01-01', 'YYYY-MM-dd'); @@ -173,7 +174,15 @@ select to_timestamp('1969 1 1', 'YYYY M u'); select to_timestamp('1969 1 6 1', 'yyyy M d u'); select to_timestamp('1969 1 5 1', 'yyyy M d u'); -- YYYY-ww-dd -select to_timestamp('1969 11 11', 'YYYY ww dd'); +select to_timestamp('2018 11 17', 'YYYY ww dd'); select to_timestamp('1969 2 11', 'YYYY W dd'); select to_timestamp('1969 5 11 11', 'YYYY M ww dd'); +select to_timestamp('2020 1 3 2', 'yyyy M w u'); +select to_timestamp('2018-11-2-17', 'yyyy-ww-W-dd'); +select to_timestamp('2018-11-2-11', 'yyyy-ww-W-dd'); +select to_timestamp('2018-11-3-12', 'yyyy-ww-W-dd'); +select to_timestamp('2018-11-10', 'yyyy-ww-dd'); +select to_timestamp('1', 'u'); +select to_timestamp('5 2', 'u d'); +select to_timestamp('5 3', 'u d'); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index a1901cd0602f2..12dbb7d2ad453 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 129 +-- Number of queries: 138 -- !query @@ -1001,12 +1001,20 @@ org.apache.spark.SparkUpgradeException You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +-- !query +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') +-- !query schema +struct +-- !query output +2017-12-31 13:33:33 + + -- !query select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -1014,7 +1022,7 @@ select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -1022,7 +1030,7 @@ select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -2017-12-25 00:00:00 +2017-12-31 00:00:00 -- !query @@ -1030,7 +1038,7 @@ select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema struct -- !query output -2017-12-25 13:33:33 +2017-12-31 13:33:33 -- !query @@ -1038,16 +1046,15 @@ select to_timestamp('1969 1 1', 'yyyy w u') -- !query schema struct -- !query output -NULL +1968-12-29 00:00:00 -- !query select to_timestamp('1969 1 1', 'yyyy M u') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +1968-12-29 00:00:00 -- !query @@ -1055,7 +1062,7 @@ select to_timestamp('1 1969 1', 'M YYYY u') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -1063,15 +1070,16 @@ select to_timestamp('1969 1 1', 'YYYY M u') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query select to_timestamp('1969 1 6 1', 'yyyy M d u') -- !query schema -struct +struct<> -- !query output -1969-01-06 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 6 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1079,15 +1087,15 @@ select to_timestamp('1969 1 5 1', 'yyyy M d u') -- !query schema struct -- !query output -NULL +1969-01-05 00:00:00 -- !query -select to_timestamp('1969 11 11', 'YYYY ww dd') +select to_timestamp('2018 11 17', 'YYYY ww dd') -- !query schema -struct +struct -- !query output -1968-12-30 00:00:00 +2018-03-11 00:00:00 -- !query @@ -1095,7 +1103,7 @@ select to_timestamp('1969 2 11', 'YYYY W dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -1103,4 +1111,69 @@ select to_timestamp('1969 5 11 11', 'YYYY M ww dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1969-03-09 00:00:00 + + +-- !query +select to_timestamp('2020 1 3 2', 'yyyy M w u') +-- !query schema +struct +-- !query output +2020-01-13 00:00:00 + + +-- !query +select to_timestamp('2018-11-2-17', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-2-11', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-3-12', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-10', 'yyyy-ww-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1', 'u') +-- !query schema +struct +-- !query output +1969-12-28 00:00:00 + + +-- !query +select to_timestamp('5 2', 'u d') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '5 2' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('5 3', 'u d') +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 85bb6a2eda7c5..7c0c0eead4316 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 129 +-- Number of queries: 138 -- !query @@ -958,12 +958,20 @@ struct> {"date":2015-10-26} +-- !query +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') +-- !query schema +struct +-- !query output +2017-12-31 13:33:33 + + -- !query select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -971,7 +979,7 @@ select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -979,7 +987,7 @@ select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -2018-01-01 00:00:00 +2017-12-31 00:00:00 -- !query @@ -987,7 +995,7 @@ select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema struct -- !query output -2018-01-01 13:33:33 +2017-12-31 13:33:33 -- !query @@ -1039,11 +1047,11 @@ NULL -- !query -select to_timestamp('1969 11 11', 'YYYY ww dd') +select to_timestamp('2018 11 17', 'YYYY ww dd') -- !query schema -struct +struct -- !query output -1969-03-10 00:00:00 +2018-03-11 00:00:00 -- !query @@ -1059,4 +1067,68 @@ select to_timestamp('1969 5 11 11', 'YYYY M ww dd') -- !query schema struct -- !query output -1969-03-10 00:00:00 +1969-03-09 00:00:00 + + +-- !query +select to_timestamp('2020 1 3 2', 'yyyy M w u') +-- !query schema +struct +-- !query output +2020-01-14 00:00:00 + + +-- !query +select to_timestamp('2018-11-2-17', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-2-11', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-3-12', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-10', 'yyyy-ww-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1', 'u') +-- !query schema +struct +-- !query output +1970-01-05 00:00:00 + + +-- !query +select to_timestamp('5 2', 'u d') +-- !query schema +struct +-- !query output +1970-01-02 00:00:00 + + +-- !query +select to_timestamp('5 3', 'u d') +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index bcdd73d1c2197..46a1e8af3bdd5 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 129 +-- Number of queries: 138 -- !query @@ -973,12 +973,20 @@ org.apache.spark.SparkUpgradeException You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +-- !query +select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') +-- !query schema +struct +-- !query output +2017-12-31 13:33:33 + + -- !query select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -986,7 +994,7 @@ select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -994,7 +1002,7 @@ select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema struct -- !query output -2017-12-25 00:00:00 +2017-12-31 00:00:00 -- !query @@ -1002,7 +1010,7 @@ select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema struct -- !query output -2017-12-25 13:33:33 +2017-12-31 13:33:33 -- !query @@ -1010,16 +1018,15 @@ select to_timestamp('1969 1 1', 'yyyy w u') -- !query schema struct -- !query output -NULL +1968-12-29 00:00:00 -- !query select to_timestamp('1969 1 1', 'yyyy M u') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +1968-12-29 00:00:00 -- !query @@ -1027,7 +1034,7 @@ select to_timestamp('1 1969 1', 'M YYYY u') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -1035,15 +1042,16 @@ select to_timestamp('1969 1 1', 'YYYY M u') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query select to_timestamp('1969 1 6 1', 'yyyy M d u') -- !query schema -struct +struct<> -- !query output -1969-01-06 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 6 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1051,15 +1059,15 @@ select to_timestamp('1969 1 5 1', 'yyyy M d u') -- !query schema struct -- !query output -NULL +1969-01-05 00:00:00 -- !query -select to_timestamp('1969 11 11', 'YYYY ww dd') +select to_timestamp('2018 11 17', 'YYYY ww dd') -- !query schema -struct +struct -- !query output -1968-12-30 00:00:00 +2018-03-11 00:00:00 -- !query @@ -1067,7 +1075,7 @@ select to_timestamp('1969 2 11', 'YYYY W dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1968-12-29 00:00:00 -- !query @@ -1075,4 +1083,69 @@ select to_timestamp('1969 5 11 11', 'YYYY M ww dd') -- !query schema struct -- !query output -1968-12-30 00:00:00 +1969-03-09 00:00:00 + + +-- !query +select to_timestamp('2020 1 3 2', 'yyyy M w u') +-- !query schema +struct +-- !query output +2020-01-13 00:00:00 + + +-- !query +select to_timestamp('2018-11-2-17', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-2-11', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-3-12', 'yyyy-ww-W-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2018-11-10', 'yyyy-ww-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1', 'u') +-- !query schema +struct +-- !query output +1969-12-28 00:00:00 + + +-- !query +select to_timestamp('5 2', 'u d') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '5 2' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('5 3', 'u d') +-- !query schema +struct +-- !query output +NULL From 0dcd5dffb8c7b74d3570c67aae2639332cebb687 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 1 Jun 2020 21:49:56 +0800 Subject: [PATCH 5/5] forbid mixed patterns in formatter --- .../util/DateTimeFormatterHelper.scala | 114 ++++++------------ .../expressions/DateExpressionsSuite.scala | 40 ++---- .../sql-tests/results/ansi/datetime.sql.out | 69 ++++++----- .../sql-tests/results/datetime.sql.out | 69 ++++++----- 4 files changed, 127 insertions(+), 165 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 52e6051559a21..f696fb17d9897 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -23,8 +23,6 @@ import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverSt import java.time.temporal._ import java.util.Locale -import scala.util.control.NonFatal - import com.google.common.cache.CacheBuilder import org.apache.spark.SparkUpgradeException @@ -33,91 +31,51 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ trait DateTimeFormatterHelper { - private def getFieldValue( - accessor: TemporalAccessor, - field: TemporalField): Option[Int] = { + + private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = { if (accessor.isSupported(field)) { - try { - Option(accessor.get(field)) - } catch { - case NonFatal(_) => None - } + accessor.get(field) } else { - None + default } } + private def mayNonWeekBased(accessor: TemporalAccessor): Boolean = { + val has = accessor.isSupported _ + has(ChronoField.YEAR) || has(ChronoField.MONTH_OF_YEAR) || has(ChronoField.DAY_OF_MONTH) + } + + private def mayWeekBased(accessor: TemporalAccessor, wf: WeekFields): Boolean = { + val has = accessor.isSupported _ + has(wf.weekBasedYear) || has(wf.weekOfMonth) || has(wf.weekOfWeekBasedYear) || has(wf.dayOfWeek) + } + @throws[DateTimeException] protected def toLocalDate(accessor: TemporalAccessor, locale: Locale): LocalDate = { val localDate = accessor.query(TemporalQueries.localDate()) // If all the date fields are resolved(yMd or Ywu), return the local date directly. if (localDate != null) return localDate - var res = LocalDate.of(1970, 1, 1) - val weekFields = WeekFields.of(locale) - val weekBasedYearField = weekFields.weekBasedYear - var weekBasedYearEnabled = false - - val year = getFieldValue(accessor, weekBasedYearField).map { y => - weekBasedYearEnabled = true - y - }.orElse { - getFieldValue(accessor, ChronoField.YEAR) - } - - val week = getFieldValue(accessor, weekFields.weekOfWeekBasedYear()) - val dayOfWeek = getFieldValue(accessor, weekFields.dayOfWeek()) - - // TODO: How to check 'W' week-of-month field, not like other week-based field, it always throw - // UnsupportedTemporalTypeException to get it even `accessor.isSupported` passed. - - if (weekBasedYearEnabled) { - // If the week-based-year field exists, only the week-based fields matters. - res.`with`(weekFields.weekOfWeekBasedYear, week.getOrElse(1).toLong) - .`with`(weekFields.dayOfWeek(), dayOfWeek.getOrElse(1).toLong) - .`with`(weekBasedYearField, year.get) + val weekBased = mayWeekBased(accessor, weekFields) + if (weekBased && mayNonWeekBased(accessor)) { + throw new DateTimeException( + s"Can not mix week-based and non-week-based date fields together for parsing dates") + } else if (weekBased) { + + val weekBasedYear = getOrDefault(accessor, weekFields.weekBasedYear, 1970) + val week = getOrDefault(accessor, weekFields.weekOfWeekBasedYear(), 1) + val dayOfWeek = getOrDefault(accessor, weekFields.dayOfWeek(), 1) + + LocalDate.of(1970, 1, 1) + .`with`(weekFields.weekOfWeekBasedYear, week) + .`with`(weekFields.dayOfWeek, dayOfWeek) + .`with`(weekFields.weekBasedYear, weekBasedYear) } else { - if (year.isDefined) { - res = res.withYear(year.get) - } - - val month = getFieldValue(accessor, ChronoField.MONTH_OF_YEAR) - val day = getFieldValue(accessor, ChronoField.DAY_OF_MONTH) - - if (month.isDefined && week.isDefined) { - // check the week fall into the correct month of the year - res = res.`with`(weekFields.weekOfWeekBasedYear(), week.get) - if (res.getMonthValue != month.get) { - throw new DateTimeException( - s"week-of-week-based-year value: ${week.get} conflicts with month-of-year value:" + - s" ${month.get} which should be ${res.getMonthValue} instead.") - } - } else if (month.isDefined) { - res = res.withMonth(month.get) - } else if (week.isDefined) { - if (day.isDefined) { - throw new DateTimeException( - s"Can not use week-of-week-based-year and day-of-month together in non-week-based" + - s" mode.") - } - res = res.`with`(weekFields.weekOfWeekBasedYear, week.get) - } - - if (dayOfWeek.isDefined && day.isDefined) { - // check whether the days matches - res = res.`with`(weekFields.dayOfWeek(), dayOfWeek.get) - if (res.getDayOfMonth != day.get) { - throw new DateTimeException( - s"day-of-week value: ${dayOfWeek.get} conflicts with day-of-month value:" + - s" ${day.get} which should be ${res.getDayOfMonth} instead.") - } - } else if (day.isDefined) { - res = res.withDayOfMonth(day.get) - } else if (dayOfWeek.isDefined) { - res = res.`with`(weekFields.dayOfWeek(), dayOfWeek.get) - } - res + val year = getOrDefault(accessor, ChronoField.YEAR, 1970) + val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1) + val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1) + LocalDate.of(year, month, day) } } @@ -134,10 +92,10 @@ trait DateTimeFormatterHelper { } else { 0 } - val minute = getFieldValue(accessor, ChronoField.MINUTE_OF_HOUR) - val second = getFieldValue(accessor, ChronoField.SECOND_OF_MINUTE) - val nanoSecond = getFieldValue(accessor, ChronoField.NANO_OF_SECOND) - LocalTime.of(hour, minute.getOrElse(0), second.getOrElse(0), nanoSecond.getOrElse(0)) + val minute = getOrDefault(accessor, ChronoField.MINUTE_OF_HOUR, 0) + val second = getOrDefault(accessor, ChronoField.SECOND_OF_MINUTE, 0) + val nanoSecond = getOrDefault(accessor, ChronoField.NANO_OF_SECOND, 0) + LocalTime.of(hour, minute, second, nanoSecond) } // Converts the parsed temporal object to ZonedDateTime. It sets time components to zeros diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 728e349f28bff..e73cf0cb1897f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1170,16 +1170,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-31868: Restore the behaviour week-based-year for 2.4") { - // TODO: Locale.US is Sunday started day of week, which affects the new formatter, - // while in 2.4 it's Monday first. - checkEvaluation( - new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child, - Timestamp.valueOf("2017-12-31 00:00:00.0")) - - checkEvaluation( - new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, - Timestamp.valueOf("2017-12-31 13:33:33.0")) - checkEvaluation( new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child, Timestamp.valueOf("1968-12-30 00:00:00.0")) @@ -1187,46 +1177,34 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // the existence of 'W' is not for generating the timestamp but likely for checking whether // the timestamp falling into it. checkEvaluation( - new ParseToTimestamp(Literal("1969 5 1 2"), Literal("YYYY W w u")).child, + new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child, Timestamp.valueOf("1968-12-30 00:00:00.0")) checkEvaluation( new ParseToTimestamp(Literal("1969 5 2"), Literal("YYYY W u")).child, Timestamp.valueOf("1968-12-30 00:00:00.0")) -// // the legacy parser does not support 'W' and results null, so SparkUpgradeException will come -// checkExceptionInExpression[SparkUpgradeException]( -// new ParseToTimestamp(Literal("1969 4 2"), Literal("YYYY W u")).child, "3.0") -// checkExceptionInExpression[SparkUpgradeException]( -// new ParseToTimestamp(Literal("5"), Literal("W")).child, "3.0") - + checkEvaluation( + new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-dd")).child, + null) // https://bugs.openjdk.java.net/browse/JDK-8145633 // Adjacent value parsing not supported for Localized Patterns checkExceptionInExpression[SparkUpgradeException]( new ParseToTimestamp(Literal("196940"), Literal("YYYYww")).child, "3.0") - checkEvaluation( - new ParseToTimestamp(Literal("2020 1 3 2"), Literal("yyyy M w u")).child, - Timestamp.valueOf("2020-01-13 00:00:00.0")) + checkExceptionInExpression[SparkUpgradeException]( + new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child, + "3.0") checkEvaluation( new ParseToTimestamp(Literal("2018-46-7 13:33:33"), Literal("YYYY-ww-u HH:mm:ss")).child, Timestamp.valueOf("2018-11-17 13:33:33.0")) - checkEvaluation( - new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, - Timestamp.valueOf("2018-03-11 00:00:00.0")) - - checkEvaluation( - new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-dd")).child, - null) + checkExceptionInExpression[SparkUpgradeException]( + new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, "3.0") - // problem of first day of week change checkExceptionInExpression[SparkUpgradeException]( new ParseToTimestamp(Literal("1969 1 6 1"), Literal("yyyy M d u")).child, "3.0") - checkEvaluation( - new ParseToTimestamp(Literal("1969 5 11 11"), Literal("YYYY M ww dd")).child, - Timestamp.valueOf("1969-03-09 00:00:00.0")) } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 30142158c7686..db8a392ee1e56 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1030,41 +1030,46 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema -struct +struct<> -- !query output -2017-12-31 13:33:33 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-11-17 13:33:33' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969-01-01' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969-12-31' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema -struct +struct<> -- !query output -2017-12-31 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-01-01' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema -struct +struct<> -- !query output -2017-12-31 13:33:33 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-11-17 13:33:33' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1072,31 +1077,34 @@ select to_timestamp('1969 1 1', 'yyyy w u') -- !query schema struct -- !query output -1968-12-29 00:00:00 +NULL -- !query select to_timestamp('1969 1 1', 'yyyy M u') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1 1969 1', 'M YYYY u') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1 1969 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1969 1 1', 'YYYY M u') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1119,9 +1127,10 @@ struct -- !query select to_timestamp('2018 11 17', 'YYYY ww dd') -- !query schema -struct +struct<> -- !query output -2018-03-11 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018 11 17' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1129,23 +1138,25 @@ select to_timestamp('1969 2 11', 'YYYY W dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +NULL -- !query select to_timestamp('1969 5 11 11', 'YYYY M ww dd') -- !query schema -struct +struct<> -- !query output -1969-03-09 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 5 11 11' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2020 1 3 2', 'yyyy M w u') -- !query schema -struct +struct<> -- !query output -2020-01-13 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 1 3 2' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1208,14 +1219,16 @@ NULL -- !query select to_timestamp('2020 2020', 'YYYY yyyy') -- !query schema -struct +struct<> -- !query output -2019-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 2020' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2020 2020', 'YYYY yyyy') -- !query schema -struct +struct<> -- !query output -2019-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 2020' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index e04e1bc98d80b..5ea43d0915b61 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1002,41 +1002,46 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema -struct +struct<> -- !query output -2017-12-31 13:33:33 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-11-17 13:33:33' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1969-01-01', 'YYYY-MM-dd') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969-01-01' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1969-12-31', 'YYYY-MM-dd') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969-12-31' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2018-01-01', 'YYYY-MM-dd') -- !query schema -struct +struct<> -- !query output -2017-12-31 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-01-01' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss') -- !query schema -struct +struct<> -- !query output -2017-12-31 13:33:33 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-11-17 13:33:33' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1044,31 +1049,34 @@ select to_timestamp('1969 1 1', 'yyyy w u') -- !query schema struct -- !query output -1968-12-29 00:00:00 +NULL -- !query select to_timestamp('1969 1 1', 'yyyy M u') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1 1969 1', 'M YYYY u') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1 1969 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('1969 1 1', 'YYYY M u') -- !query schema -struct +struct<> -- !query output -1968-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1091,9 +1099,10 @@ struct -- !query select to_timestamp('2018 11 17', 'YYYY ww dd') -- !query schema -struct +struct<> -- !query output -2018-03-11 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018 11 17' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1101,23 +1110,25 @@ select to_timestamp('1969 2 11', 'YYYY W dd') -- !query schema struct -- !query output -1968-12-29 00:00:00 +NULL -- !query select to_timestamp('1969 5 11 11', 'YYYY M ww dd') -- !query schema -struct +struct<> -- !query output -1969-03-09 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 5 11 11' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2020 1 3 2', 'yyyy M w u') -- !query schema -struct +struct<> -- !query output -2020-01-13 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 1 3 2' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -1180,14 +1191,16 @@ NULL -- !query select to_timestamp('2020 2020', 'YYYY yyyy') -- !query schema -struct +struct<> -- !query output -2019-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 2020' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query select to_timestamp('2020 2020', 'YYYY yyyy') -- !query schema -struct +struct<> -- !query output -2019-12-29 00:00:00 +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 2020' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.