diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index f0ed89ea1f..0ebeacb41f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1008,17 +1008,17 @@ case class UnixTimestamp( copy(timeExp = newLeft, format = newRight) } -case class GetTimestampNTZ( +/** + * Gets a timestamp from a string or a date. + */ +case class GetTimestamp( left: Expression, right: Expression, + override val dataType: DataType, timeZoneId: Option[String] = None, failOnError: Boolean = SQLConf.get.ansiEnabled) extends ToTimestamp { - override val forTimestampNTZ: Boolean = true - - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType) - - override def dataType: DataType = TimestampNTZType + override val forTimestampNTZ: Boolean = dataType == TimestampNTZType override protected def downScaleFactor: Long = 1 @@ -1064,7 +1064,7 @@ case class ParseToTimestampNTZ( child: Expression) extends RuntimeReplaceable { def this(left: Expression, format: Expression) = { - this(left, Option(format), GetTimestampNTZ(left, format)) + this(left, Option(format), GetTimestamp(left, format, TimestampNTZType)) } def this(left: Expression) = this(left, None, Cast(left, TimestampNTZType)) @@ -1886,7 +1886,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr extends RuntimeReplaceable { def this(left: Expression, format: Expression) = { - this(left, Option(format), Cast(GetTimestamp(left, format), DateType)) + this(left, Option(format), Cast(GetTimestamp(left, format, TimestampType), DateType)) } def this(left: Expression) = { @@ -1911,7 +1911,8 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr usage = """ _FUNC_(timestamp_str[, fmt]) - Parses the `timestamp_str` expression with the `fmt` expression to a timestamp. Returns null with invalid input. By default, it follows casting rules to - a timestamp if the `fmt` is omitted. + a timestamp if the `fmt` is omitted. The result data type is consistent with the value of + configuration `spark.sql.timestampType`. """, arguments = """ Arguments: @@ -1929,20 +1930,24 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr group = "datetime_funcs", since = "2.2.0") // scalastyle:on line.size.limit -case class ParseToTimestamp(left: Expression, format: Option[Expression], child: Expression) - extends RuntimeReplaceable { +case class ParseToTimestamp( + left: Expression, + format: Option[Expression], + override val dataType: DataType, + child: Expression) extends RuntimeReplaceable { def this(left: Expression, format: Expression) = { - this(left, Option(format), GetTimestamp(left, format)) + this(left, Option(format), SQLConf.get.timestampType, + GetTimestamp(left, format, SQLConf.get.timestampType)) } - def this(left: Expression) = this(left, None, Cast(left, TimestampType)) + def this(left: Expression) = + this(left, None, SQLConf.get.timestampType, Cast(left, SQLConf.get.timestampType)) override def flatArguments: Iterator[Any] = Iterator(left, format) override def exprsReplaced: Seq[Expression] = left +: format.toSeq override def prettyName: String = "to_timestamp" - override def dataType: DataType = TimestampType override protected def withNewChildInternal(newChild: Expression): ParseToTimestamp = copy(child = newChild) @@ -2198,27 +2203,6 @@ case class DateDiff(endDate: Expression, startDate: Expression) copy(endDate = newLeft, startDate = newRight) } -/** - * Gets timestamps from strings using given pattern. - */ -private case class GetTimestamp( - left: Expression, - right: Expression, - timeZoneId: Option[String] = None, - failOnError: Boolean = SQLConf.get.ansiEnabled) - extends ToTimestamp { - - override val downScaleFactor = 1 - override def dataType: DataType = TimestampType - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override protected def withNewChildrenInternal( - newLeft: Expression, newRight: Expression): GetTimestamp = - copy(left = newLeft, right = newRight) -} - @ExpressionDescription( usage = "_FUNC_(year, month, day) - Create date from year, month and day fields.", arguments = """ 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 02d6d95399..f288d82016 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 @@ -1427,12 +1427,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val input = s.replace("T", " ") val expectedTs = LocalDateTime.parse(s) checkEvaluation( - GetTimestampNTZ(Literal(input), Literal("yyyy-MM-dd HH:mm:ss")), expectedTs) + GetTimestamp(Literal(input), Literal("yyyy-MM-dd HH:mm:ss"), TimestampNTZType), + expectedTs) Seq(".123456", ".123456PST", ".123456CST", ".123456UTC").foreach { segment => val input2 = input + segment val expectedTs2 = LocalDateTime.parse(s + ".123456") checkEvaluation( - GetTimestampNTZ(Literal(input2), Literal("yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]")), + GetTimestamp(Literal(input2), Literal("yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]"), + TimestampNTZType), expectedTs2) } } @@ -1446,19 +1448,22 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( GetTimestamp( Literal("2020-01-27T20:06:11.847-0800"), - Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz")), 1580184371847000L) + Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz"), TimestampType), + 1580184371847000L) } withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "corrected") { checkEvaluation( GetTimestamp( Literal("2020-01-27T20:06:11.847-0800"), - Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz")), null) + Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz"), + TimestampType), null) } withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "exception") { checkExceptionInExpression[SparkUpgradeException]( GetTimestamp( Literal("2020-01-27T20:06:11.847-0800"), - Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz")), "Fail to parse") + Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz"), + TimestampType), "Fail to parse") } } @@ -1684,8 +1689,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) { val exprSeq = Seq[Expression]( - GetTimestamp(Literal("2020-01-27T20:06:11.847"), Literal("yyyy-MM-dd HH:mm:ss.SSS")), - GetTimestamp(Literal("Unparseable"), Literal("yyyy-MM-dd HH:mm:ss.SSS")), + GetTimestamp(Literal("2020-01-27T20:06:11.847"), Literal("yyyy-MM-dd HH:mm:ss.SSS"), + TimestampType), + GetTimestamp(Literal("Unparseable"), Literal("yyyy-MM-dd HH:mm:ss.SSS"), + TimestampType), UnixTimestamp(Literal("2020-01-27T20:06:11.847"), Literal("yyyy-MM-dd HH:mm:ss.SSS")), UnixTimestamp(Literal("Unparseable"), Literal("yyyy-MM-dd HH:mm:ss.SSS")), ToUnixTimestamp(Literal("2020-01-27T20:06:11.847"), Literal("yyyy-MM-dd HH:mm:ss.SSS")), @@ -1704,7 +1711,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // LEGACY works, CORRECTED failed, EXCEPTION with SparkUpgradeException val exprSeq2 = Seq[(Expression, Long)]( (GetTimestamp(Literal("2020-01-27T20:06:11.847!!!"), - Literal("yyyy-MM-dd'T'HH:mm:ss.SSS")), 1580184371847000L), + Literal("yyyy-MM-dd'T'HH:mm:ss.SSS"), TimestampType), 1580184371847000L), (UnixTimestamp(Literal("2020-01-27T20:06:11.847!!!"), Literal("yyyy-MM-dd'T'HH:mm:ss.SSS")), 1580184371L), (ToUnixTimestamp(Literal("2020-01-27T20:06:11.847!!!"), diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out index 3905acdb89..ce406e2b72 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out @@ -146,7 +146,7 @@ NULL 2016-12-31 2016-12-31 -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct +struct -- !query output NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 @@ -708,7 +708,7 @@ struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct -- !query output NULL @@ -716,7 +716,7 @@ NULL -- !query select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12 @@ -724,7 +724,7 @@ struct +struct -- !query output 2019-10-06 10:11:12.1 @@ -732,7 +732,7 @@ struct +struct -- !query output 2019-10-06 10:11:12.12 @@ -740,15 +740,15 @@ struct +struct -- !query output -2019-10-06 03:11:12.123 +2019-10-06 10:11:12.123 -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12.1234 @@ -756,15 +756,15 @@ struct +struct -- !query output -2019-10-06 08:11:12.12345 +2019-10-06 10:11:12.12345 -- !query select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12.123456 @@ -772,7 +772,7 @@ struct +struct -- !query output NULL @@ -780,7 +780,7 @@ NULL -- !query select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12.123456 @@ -788,7 +788,7 @@ struct +struct -- !query output NULL @@ -796,7 +796,7 @@ NULL -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12.1234 @@ -804,7 +804,7 @@ struct +struct -- !query output 2019-10-06 10:11:12.123 @@ -812,7 +812,7 @@ struct +struct -- !query output 2019-10-06 10:11:12 @@ -820,7 +820,7 @@ struct +struct -- !query output 2019-10-06 10:11:12.12 @@ -828,7 +828,7 @@ struct +struct -- !query output 2019-10-06 10:11:00 @@ -836,7 +836,7 @@ struct -- !query select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12.12345 @@ -844,7 +844,7 @@ struct +struct -- !query output 2019-10-06 10:11:12.1234 @@ -852,7 +852,7 @@ struct +struct -- !query output NULL @@ -860,7 +860,7 @@ NULL -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query schema -struct +struct -- !query output NULL @@ -868,7 +868,7 @@ NULL -- !query select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") -- !query schema -struct +struct -- !query output 0019-10-06 10:11:12.1234 @@ -876,7 +876,7 @@ struct -- !query select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") -- !query schema -struct +struct -- !query output 2019-10-06 00:00:00 @@ -884,7 +884,7 @@ struct -- !query select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") -- !query schema -struct +struct -- !query output 2019-10-06 00:00:00 @@ -892,7 +892,7 @@ struct -- !query select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12.12 @@ -900,7 +900,7 @@ struct +struct -- !query output 2019-10-06 10:11:12 @@ -908,7 +908,7 @@ struct -- !query select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12 @@ -916,7 +916,7 @@ struct -- !query select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") -- !query schema -struct +struct -- !query output 2019-10-06 10:11:12 @@ -1140,7 +1140,7 @@ struct +struct -- !query output 1970-01-16 00:00:00 @@ -1148,7 +1148,7 @@ struct -- !query select to_timestamp("02-29", "MM-dd") -- !query schema -struct +struct -- !query output NULL @@ -1188,7 +1188,7 @@ NULL -- !query select to_timestamp("2019 40", "yyyy mm") -- !query schema -struct +struct -- !query output 2019-01-01 00:40:00 @@ -1196,7 +1196,7 @@ struct -- !query select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") -- !query schema -struct +struct -- !query output 2019-01-01 10:10:10 @@ -1222,8 +1222,8 @@ select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !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 recognize 'yyyy-MM-dd GGGGG' 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 +java.lang.RuntimeException +Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -1231,8 +1231,8 @@ select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') -- !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 recognize 'dd MM yyyy EEEEEE' 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 +java.lang.RuntimeException +Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -1240,8 +1240,8 @@ select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !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 recognize 'dd MM yyyy EEEEE' 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 +java.lang.RuntimeException +Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -1334,7 +1334,7 @@ NULL -- !query select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct -- !query output NULL @@ -1342,7 +1342,7 @@ NULL -- !query select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct -- !query output NULL