From f39dd48ffea3ce39c6a2659d3c690a5116717a9a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 May 2020 11:45:34 +0800 Subject: [PATCH 01/11] [SPARK-31830][SQL] Consistent error handling for datetime formatting functions --- docs/sql-migration-guide.md | 2 + .../expressions/datetimeExpressions.scala | 100 ++++-------------- .../expressions/DateExpressionsSuite.scala | 37 ++++++- .../resources/sql-tests/inputs/datetime.sql | 4 + .../sql-tests/results/ansi/datetime.sql.out | 29 ++++- .../sql-tests/results/datetime-legacy.sql.out | 29 ++++- .../sql-tests/results/datetime.sql.out | 29 ++++- 7 files changed, 141 insertions(+), 89 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 2272c9038484..8410a57f2c4e 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -27,6 +27,8 @@ license: | - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. + + - In Spark 3.1, `from_unixtime` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, it results `NULL`. ## Upgrading from Spark SQL 2.4 to 3.0 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 7dc008a2e5df..35e5b41b288e 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 @@ -34,7 +34,6 @@ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, LegacyDateFormats, Tim import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.SIMPLE_DATE_FORMAT -import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -1053,91 +1052,38 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) - private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: TimestampFormatter = - try { - TimestampFormatter( - constFormat.toString, - zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, + private lazy val formatter: Option[TimestampFormatter] = + if (right.foldable) { + Option(right.eval()).map { format => + TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT, needVarLengthSecondFraction = false) - } catch { - case e: SparkUpgradeException => throw e - case NonFatal(_) => null } + } else None - override def eval(input: InternalRow): Any = { - val time = left.eval(input) - if (time == null) { - null - } else { - if (format.foldable) { - if (constFormat == null || formatter == null) { - null - } else { - try { - UTF8String.fromString(formatter.format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) - } catch { - case e: SparkUpgradeException => throw e - case NonFatal(_) => null - } - } - } else { - val f = format.eval(input) - if (f == null) { - null - } else { - try { - UTF8String.fromString( - TimestampFormatter( - f.toString, - zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, - needVarLengthSecondFraction = false) - .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) - } catch { - case e: SparkUpgradeException => throw e - case NonFatal(_) => null - } - } - } - } + override def nullSafeEval(seconds: Any, format: Any): Any = { + val ft = formatter.getOrElse(TimestampFormatter(format.toString, zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, needVarLengthSecondFraction = false)) + UTF8String.fromString(ft.format(seconds.asInstanceOf[Long] * MICROS_PER_SECOND)) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val df = classOf[TimestampFormatter].getName - if (format.foldable) { - if (formatter == null) { - ExprCode.forNullValue(StringType) - } else { - val formatterName = ctx.addReferenceObj("formatter", formatter, df) - val t = left.genCode(ctx) - ev.copy(code = code""" - ${t.code} - boolean ${ev.isNull} = ${t.isNull}; - ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; - if (!${ev.isNull}) { - try { - ${ev.value} = UTF8String.fromString($formatterName.format(${t.value} * 1000000L)); - } catch (java.lang.IllegalArgumentException e) { - ${ev.isNull} = true; - } - }""") - } - } else { - val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + formatter.map { f => + val formatterName = ctx.addReferenceObj("formatter", f) + defineCodeGen(ctx, ev, (seconds, _) => + s"UTF8String.fromString($formatterName.format($seconds * 1000000L))") + }.getOrElse { val tf = TimestampFormatter.getClass.getName.stripSuffix("$") val ldf = LegacyDateFormats.getClass.getName.stripSuffix("$") - nullSafeCodeGen(ctx, ev, (seconds, f) => { + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + defineCodeGen(ctx, ev, (seconds, format) => s""" - try { - ${ev.value} = UTF8String.fromString( - $tf$$.MODULE$$.apply($f.toString(), $zid, $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), false) - .format($seconds * 1000000L)); - } catch (java.lang.IllegalArgumentException e) { - ${ev.isNull} = true; - }""" - }) + |UTF8String.fromString( + | $tf$$.MODULE$$.apply($format.toString(), + | $zid, + | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), + | false) + | .format($seconds * 1000000L)) + |""".stripMargin) } } } 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 02d6d847dc06..cbc141a3bdf9 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.time.{Instant, LocalDate, ZoneId} +import java.time.format.DateTimeFormatter import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit._ @@ -777,8 +778,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), null) - checkEvaluation( - FromUnixTime(Literal(0L), Literal("not a valid format"), timeZoneId), null) // SPARK-28072 The codegen path for non-literal input should also work checkEvaluation( @@ -792,7 +791,39 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } // Test escaping of format - GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\"quote")) :: Nil) + val e = FromUnixTime(Literal(0L), Literal("\"")) + GenerateUnsafeProjection.generate(e.withTimeZone(conf.sessionLocalTimeZone) :: Nil) + } + + test("from_unixtime with invalid datetime pattern") { + val invalidForBoth = Seq("A", "c", "n", "e", "n", "p") + val invalidForNew = Seq("MMMMM", "GGGGG") + + invalidForBoth.foreach { format => + Seq("exception", "legacy", "corrected").foreach { policy => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> policy) { + checkExceptionInExpression[IllegalArgumentException]( + FromUnixTime(Literal(0L), Literal(format)), s"${format.head}") + } + } + } + + invalidForNew.foreach { format => + Seq("exception", "corrected").foreach { policy => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> policy) { + checkExceptionInExpression[SparkUpgradeException]( + FromUnixTime(Literal(0L), Literal(format)), s"${format.head}") + } + } + } + + invalidForNew.foreach { format => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "legacy") { + checkEvaluation( + FromUnixTime(Literal(0L), Literal(format)), + new SimpleDateFormat(format).format(new Date(0))) + } + } } test("unix_timestamp") { 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 663c62f1a6f6..96f4eab9ea71 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,7 @@ 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')); + +select from_unixtime(a, b) from + values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b); +select from_unixtime(12345, 'invalid'); 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 5857a0ac90c7..3d35732b74a5 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: 118 -- !query @@ -951,9 +951,10 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query select from_unixtime(54321, 'QQQQQ') -- !query schema -struct +struct<> -- !query output -NULL +java.lang.IllegalArgumentException +Too many pattern letters: Q -- !query @@ -999,3 +1000,25 @@ 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 from_unixtime(a, b) from + values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b) +-- !query schema +struct +-- !query output +1970-01-01 +NULL +NULL +NULL +NULL + + +-- !query +select from_unixtime(12345, 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n 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 8a726efafad8..46fd2a0c6790 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: 118 -- !query @@ -913,9 +913,10 @@ December -- !query select from_unixtime(54321, 'QQQQQ') -- !query schema -struct +struct<> -- !query output -NULL +java.lang.IllegalArgumentException +Illegal pattern character 'Q' -- !query @@ -956,3 +957,25 @@ select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy struct> -- !query output {"date":2015-10-26} + + +-- !query +select from_unixtime(a, b) from + values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b) +-- !query schema +struct +-- !query output +1970-01-01 +NULL +NULL +NULL +NULL + + +-- !query +select from_unixtime(12345, 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' 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 7cacaec42c81..edf29262b06c 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: 118 -- !query @@ -923,9 +923,10 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query select from_unixtime(54321, 'QQQQQ') -- !query schema -struct +struct<> -- !query output -NULL +java.lang.IllegalArgumentException +Too many pattern letters: Q -- !query @@ -971,3 +972,25 @@ 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 from_unixtime(a, b) from + values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b) +-- !query schema +struct +-- !query output +1970-01-01 +NULL +NULL +NULL +NULL + + +-- !query +select from_unixtime(12345, 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n From 33fedf8a5cd3001c411edb34d61df2295a2dae2b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 28 May 2020 13:47:34 +0800 Subject: [PATCH 02/11] parsing --- docs/sql-migration-guide.md | 2 +- .../expressions/datetimeExpressions.scala | 217 +++--- .../expressions/DateExpressionsSuite.scala | 89 +-- .../resources/sql-tests/inputs/datetime.sql | 86 ++- .../sql-tests/results/ansi/datetime.sql.out | 632 +++++++++++++++++- .../sql-tests/results/datetime-legacy.sql.out | 632 +++++++++++++++++- .../sql-tests/results/datetime.sql.out | 632 +++++++++++++++++- .../apache/spark/sql/DateFunctionsSuite.scala | 5 +- 8 files changed, 2048 insertions(+), 247 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 8410a57f2c4e..0130923e694b 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -28,7 +28,7 @@ license: | - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. - - In Spark 3.1, `from_unixtime` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, it results `NULL`. + - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. ## Upgrading from Spark SQL 2.4 to 3.0 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 35e5b41b288e..8872b748210a 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 @@ -17,7 +17,9 @@ package org.apache.spark.sql.catalyst.expressions +import java.text.ParseException import java.time.{DateTimeException, LocalDate, LocalDateTime, ZoneId} +import java.time.format.DateTimeParseException import java.time.temporal.IsoFields import java.util.Locale @@ -25,7 +27,6 @@ import scala.util.control.NonFatal import org.apache.commons.text.StringEscapeUtils -import org.apache.spark.SparkUpgradeException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -55,6 +56,28 @@ trait TimeZoneAwareExpression extends Expression { @transient lazy val zoneId: ZoneId = DateTimeUtils.getZoneId(timeZoneId.get) } +trait TimestampFormatterHelper extends TimeZoneAwareExpression { + + protected def formatString: Expression + + protected def isParsing: Boolean + + @transient final protected lazy val formatterOption: Option[TimestampFormatter] = + if (formatString.foldable) { + Option(formatString.eval()).map(fmt => getFormatter(fmt.toString)) + } else None + + final protected def getFormatter(fmt: String): TimestampFormatter = { + val formatter = TimestampFormatter( + format = fmt, + zoneId = zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = isParsing) + formatter.validatePatternString() + formatter + } +} + /** * Returns the current date at the start of query evaluation. * All calls of current_date within the same query return the same value. @@ -703,7 +726,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa since = "1.5.0") // scalastyle:on line.size.limit case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + extends BinaryExpression with TimestampFormatterHelper with ImplicitCastInputTypes { def this(left: Expression, right: Expression) = this(left, right, None) @@ -714,33 +737,13 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) - @transient private lazy val formatter: Option[TimestampFormatter] = { - if (right.foldable) { - Option(right.eval()).map { format => - TimestampFormatter( - format.toString, - zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, - needVarLengthSecondFraction = false) - } - } else None - } - override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val tf = if (formatter.isEmpty) { - TimestampFormatter( - format.toString, - zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, - needVarLengthSecondFraction = false) - } else { - formatter.get - } - UTF8String.fromString(tf.format(timestamp.asInstanceOf[Long])) + val formatter = formatterOption.getOrElse(getFormatter(format.toString)) + UTF8String.fromString(formatter.format(timestamp.asInstanceOf[Long])) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - formatter.map { tf => + formatterOption.map { tf => val timestampFormatter = ctx.addReferenceObj("timestampFormatter", tf) defineCodeGen(ctx, ev, (timestamp, _) => { s"""UTF8String.fromString($timestampFormatter.format($timestamp))""" @@ -761,6 +764,10 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti } override def prettyName: String = "date_format" + + override protected def formatString: Expression = right + + override protected def isParsing: Boolean = false } /** @@ -858,127 +865,80 @@ case class UnixTimestamp(timeExp: Expression, format: Expression, timeZoneId: Op } abstract class ToTimestamp - extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { + extends BinaryExpression with TimestampFormatterHelper with ExpectsInputTypes { // The result of the conversion to timestamp is microseconds divided by this factor. // For example if the factor is 1000000, the result of the expression is in seconds. protected def downScaleFactor: Long + override protected def formatString: Expression = right + override protected def isParsing = true + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, DateType, TimestampType), StringType) override def dataType: DataType = LongType override def nullable: Boolean = true - private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: TimestampFormatter = - try { - TimestampFormatter( - constFormat.toString, - zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, - needVarLengthSecondFraction = true) - } catch { - case e: SparkUpgradeException => throw e - case NonFatal(_) => null - } - override def eval(input: InternalRow): Any = { - val t = left.eval(input) - if (t == null) { - null - } else { + Option(left.eval(input)).map { t => left.dataType match { - case DateType => - epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor - case TimestampType => - t.asInstanceOf[Long] / downScaleFactor - case StringType if right.foldable => - if (constFormat == null || formatter == null) { - null - } else { - try { - formatter.parse( - t.asInstanceOf[UTF8String].toString) / downScaleFactor - } catch { - case e: SparkUpgradeException => throw e - case NonFatal(_) => null - } - } + case DateType => epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor + case TimestampType => t.asInstanceOf[Long] / downScaleFactor case StringType => - val f = right.eval(input) - if (f == null) { - null - } else { - val formatString = f.asInstanceOf[UTF8String].toString + Option(right.eval(input)).map { fmt => + val formatter = formatterOption.getOrElse(getFormatter(fmt.toString)) try { - TimestampFormatter( - formatString, - zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, - needVarLengthSecondFraction = true) - .parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor + formatter.parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor } catch { - case e: SparkUpgradeException => throw e - case NonFatal(_) => null + case _: DateTimeParseException | + _: DateTimeException | + _: ParseException => null } - } + }.orNull } - } + }.orNull } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = CodeGenerator.javaType(dataType) left.dataType match { - case StringType if right.foldable => + case StringType => formatterOption.map { fmt => val df = classOf[TimestampFormatter].getName - if (formatter == null) { - ExprCode.forNullValue(dataType) - } else { - val formatterName = ctx.addReferenceObj("formatter", formatter, df) - val eval1 = left.genCode(ctx) - ev.copy(code = code""" - ${eval1.code} - boolean ${ev.isNull} = ${eval1.isNull}; - $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; - if (!${ev.isNull}) { - try { - ${ev.value} = $formatterName.parse(${eval1.value}.toString()) / $downScaleFactor; - } catch (java.lang.IllegalArgumentException e) { - ${ev.isNull} = true; - } catch (java.text.ParseException e) { - ${ev.isNull} = true; - } catch (java.time.format.DateTimeParseException e) { - ${ev.isNull} = true; - } catch (java.time.DateTimeException e) { - ${ev.isNull} = true; - } - }""") - } - case StringType => + val formatterName = ctx.addReferenceObj("formatter", fmt, df) + nullSafeCodeGen(ctx, ev, (datetimeStr, _) => + s""" + |try { + | ${ev.value} = $formatterName.parse($datetimeStr.toString()) / $downScaleFactor; + |} catch (java.time.DateTimeException e) { + | ${ev.isNull} = true; + |} catch (java.time.format.DateTimeParseException e) { + | ${ev.isNull} = true; + |} catch (java.text.ParseException e) { + | ${ev.isNull} = true; + |} + |""".stripMargin) + }.getOrElse { val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val tf = TimestampFormatter.getClass.getName.stripSuffix("$") val ldf = LegacyDateFormats.getClass.getName.stripSuffix("$") - nullSafeCodeGen(ctx, ev, (string, format) => { + nullSafeCodeGen(ctx, ev, (string, format) => s""" - try { - ${ev.value} = $tf$$.MODULE$$.apply( - $format.toString(), - $zid, - $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), - true) - .parse($string.toString()) / $downScaleFactor; - } catch (java.lang.IllegalArgumentException e) { - ${ev.isNull} = true; - } catch (java.text.ParseException e) { - ${ev.isNull} = true; - } catch (java.time.format.DateTimeParseException e) { - ${ev.isNull} = true; - } catch (java.time.DateTimeException e) { - ${ev.isNull} = true; - } - """ - }) + |try { + | ${ev.value} = $tf$$.MODULE$$.apply( + | $format.toString(), + | $zid, + | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), + | true).parse($string.toString()) / $downScaleFactor; + |} catch (java.time.format.DateTimeParseException e) { + | ${ev.isNull} = true; + |} catch (java.time.DateTimeException e) { + | ${ev.isNull} = true; + |} catch (java.text.ParseException e) { + | ${ev.isNull} = true; + |} + |""".stripMargin) + } case TimestampType => val eval1 = left.genCode(ctx) ev.copy(code = code""" @@ -1031,7 +991,7 @@ abstract class UnixTime extends ToTimestamp { since = "1.5.0") // scalastyle:on line.size.limit case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + extends BinaryExpression with TimestampFormatterHelper with ImplicitCastInputTypes { def this(sec: Expression, format: Expression) = this(sec, format, None) @@ -1052,22 +1012,13 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) - private lazy val formatter: Option[TimestampFormatter] = - if (right.foldable) { - Option(right.eval()).map { format => - TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT, - needVarLengthSecondFraction = false) - } - } else None - override def nullSafeEval(seconds: Any, format: Any): Any = { - val ft = formatter.getOrElse(TimestampFormatter(format.toString, zoneId, - legacyFormat = SIMPLE_DATE_FORMAT, needVarLengthSecondFraction = false)) - UTF8String.fromString(ft.format(seconds.asInstanceOf[Long] * MICROS_PER_SECOND)) + val fmt = formatterOption.getOrElse(getFormatter(format.toString)) + UTF8String.fromString(fmt.format(seconds.asInstanceOf[Long] * MICROS_PER_SECOND)) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - formatter.map { f => + formatterOption.map { f => val formatterName = ctx.addReferenceObj("formatter", f) defineCodeGen(ctx, ev, (seconds, _) => s"UTF8String.fromString($formatterName.format($seconds * 1000000L))") @@ -1086,6 +1037,10 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ |""".stripMargin) } } + + override protected def formatString: Expression = format + + override protected def isParsing: Boolean = false } /** 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 7661f4dd8844..54f54a340cc6 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 @@ -791,38 +791,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } // Test escaping of format - GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\"quote"), UTC_OPT) :: Nil) - } - - test("from_unixtime with invalid datetime pattern") { - val invalidForBoth = Seq("A", "c", "n", "e", "n", "p") - val invalidForNew = Seq("MMMMM", "GGGGG") - - invalidForBoth.foreach { format => - Seq("exception", "legacy", "corrected").foreach { policy => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> policy) { - checkExceptionInExpression[IllegalArgumentException]( - FromUnixTime(Literal(0L), Literal(format)), s"${format.head}") - } - } - } - - invalidForNew.foreach { format => - Seq("exception", "corrected").foreach { policy => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> policy) { - checkExceptionInExpression[SparkUpgradeException]( - FromUnixTime(Literal(0L), Literal(format)), s"${format.head}") - } - } - } - - invalidForNew.foreach { format => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "legacy") { - checkEvaluation( - FromUnixTime(Literal(0L), Literal(format)), - new SimpleDateFormat(format).format(new Date(0))) - } - } + GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\""), UTC_OPT) :: Nil) } test("unix_timestamp") { @@ -884,15 +853,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), MICROSECONDS.toSeconds( DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) - checkEvaluation( - UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) } } } } // Test escaping of format GenerateUnsafeProjection.generate( - UnixTimestamp(Literal("2015-07-24"), Literal("\"quote"), UTC_OPT) :: Nil) + UnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) } test("to_unix_timestamp") { @@ -950,10 +917,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { Literal(date1), Literal.create(null, StringType), timeZoneId), MICROSECONDS.toSeconds( DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) - checkEvaluation( - ToUnixTimestamp( - Literal("2015-07-24"), - Literal("not a valid format"), timeZoneId), null) // SPARK-28072 The codegen path for non-literal input should also work checkEvaluation( @@ -970,7 +933,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } // Test escaping of format GenerateUnsafeProjection.generate( - ToUnixTimestamp(Literal("2015-07-24"), Literal("\"quote"), UTC_OPT) :: Nil) + ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) } test("datediff") { @@ -1198,4 +1161,50 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkExceptionInExpression[ArithmeticException]( MillisToTimestamp(Literal(-92233720368547758L)), "long overflow") } + + test("invalid datetime pattern for parsing and formatting") { + val invalidForBoth = Seq("A", "c", "n", "e", "n", "p") + val invalidForNew = Seq("MMMMM", "GGGGG") + + invalidForBoth.foreach { format => + Seq("exception", "legacy", "corrected").foreach { policy => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> policy) { + checkExceptionInExpression[IllegalArgumentException]( + DateFormatClass(CurrentTimestamp(), Literal(format)), s"${format.head}") + checkExceptionInExpression[IllegalArgumentException]( + FromUnixTime(Literal(0L), Literal(format)), s"${format.head}") + checkExceptionInExpression[IllegalArgumentException]( + UnixTimestamp(Literal(date), Literal(format)), s"${format.head}") + } + } + } + + invalidForNew.foreach { format => + Seq("exception", "corrected").foreach { policy => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> policy) { + checkExceptionInExpression[SparkUpgradeException]( + DateFormatClass(CurrentTimestamp(), Literal(format)), s"${format.head}") + checkExceptionInExpression[SparkUpgradeException]( + FromUnixTime(Literal(0L), Literal(format)), s"${format.head}") + checkExceptionInExpression[SparkUpgradeException]( + UnixTimestamp(Literal(date), Literal(format)), s"${format.head}") + checkExceptionInExpression[SparkUpgradeException]( + UnixTimestamp(Literal(date), Literal(format)), s"${format.head}") + } + } + } + + invalidForNew.foreach { format => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "legacy") { + checkEvaluation( + DateFormatClass(Literal(ts), Literal(format)), + new SimpleDateFormat(format).format(ts)) + checkEvaluation( + FromUnixTime(Literal(0L), Literal(format)), + new SimpleDateFormat(format).format(new Date(0))) + checkEvaluation(UnixTimestamp(Literal(date), Literal(format)), null) + checkEvaluation(ToUnixTimestamp(Literal(date), Literal(format)), null) + } + } + } } 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 96f4eab9ea71..118d4bffcfbb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -13,10 +13,6 @@ select TIMESTAMP_MILLIS(-92233720368547758); -- [SPARK-16836] current_date and current_timestamp literals select current_date = current_date(), current_timestamp = current_timestamp(); -select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd'); - -select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd'); - select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15'); -- [SPARK-22333]: timeFunctionCall has conflicts with columnReference @@ -161,6 +157,82 @@ select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'd 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')); -select from_unixtime(a, b) from - values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b); -select from_unixtime(12345, 'invalid'); +select date_format(null, null); +select date_format(null, 'yyyy-MM-dd'); +select date_format(null, 'invalid'); +select date_format(cast(null as date), 'yyyy-MM-dd'); +select date_format(date '1986-05-23', null); +select date_format(date '1986-05-23', 'invalid'); +select date_format(date '1986-05-23', 'yyyy-MM-dd'); +select date_format(cast(null as string), 'yyyy-MM-dd'); +select date_format('1986-05-23', null); +select date_format('1986-05-23', 'invalid'); +select date_format('1986-05-23', 'yyyy-MM-dd'); +select date_format(cast(null as timestamp ), 'yyyy-MM-dd'); +select date_format(timestamp '1986-05-23', null); +select date_format(timestamp '1986-05-23', 'invalid'); +select date_format(timestamp '1986-05-23', 'yyyy-MM-dd'); + +select from_unixtime(null); +select from_unixtime(null , null); +select from_unixtime(12345 , null); +select from_unixtime(null , 'invalid'); +select from_unixtime(null , 'yyyy-MM-dd'); +select from_unixtime(12345 , 'yyyy-MM-dd'); + +select unix_timestamp(); +select unix_timestamp(null); +select unix_timestamp(null, null); +select unix_timestamp(null, 'yyyy-MM-dd'); +select unix_timestamp(null, 'invalid'); +select unix_timestamp(cast(null as date), 'yyyy-MM-dd'); +select unix_timestamp(date '1986-05-23'); +select unix_timestamp(date '1986-05-23', null); +select unix_timestamp(date '1986-05-23', 'invalid'); +select unix_timestamp(date '1986-05-23', 'yyyy-MM-dd'); +select unix_timestamp(cast(null as string), 'yyyy-MM-dd'); +select unix_timestamp('1986-05-23'); +select unix_timestamp('1986-05-23', null); +select unix_timestamp('1986-05-23', 'invalid'); +select unix_timestamp('1986-05-23', 'yyyy-MM-dd'); +select unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd'); +select unix_timestamp(timestamp '1986-05-23'); +select unix_timestamp(timestamp '1986-05-23', null); +select unix_timestamp(timestamp '1986-05-23', 'invalid'); +select unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd'); + +select to_unix_timestamp(null); +select to_unix_timestamp(null, null); +select to_unix_timestamp(null, 'yyyy-MM-dd'); +select to_unix_timestamp(null, 'invalid'); +select to_unix_timestamp(cast(null as date), 'yyyy-MM-dd'); +select to_unix_timestamp(date '1986-05-23'); +select to_unix_timestamp(date '1986-05-23', null); +select to_unix_timestamp(date '1986-05-23', 'invalid'); +select to_unix_timestamp(date '1986-05-23', 'yyyy-MM-dd'); +select to_unix_timestamp(cast(null as string), 'yyyy-MM-dd'); +select to_unix_timestamp('1986-05-23'); +select to_unix_timestamp('1986-05-23', null); +select to_unix_timestamp('1986-05-23', 'invalid'); +select to_unix_timestamp('1986-05-23', 'yyyy-MM-dd'); +select to_unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd'); +select to_unix_timestamp(timestamp '1986-05-23'); +select to_unix_timestamp(timestamp '1986-05-23', null); +select to_unix_timestamp(timestamp '1986-05-23', 'invalid'); +select to_unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd'); + +select to_timestamp(null); +select to_timestamp(cast(null as string), 'yyyy-MM-dd'); +select to_timestamp(cast(null as string), 'invalid'); +select to_timestamp('1986-05-23'); +select to_timestamp('1986-05-23', null); +select to_timestamp('1986-05-23', 'invalid'); +select to_timestamp('1986-05-23', 'yyyy-MM-dd'); + +select to_date(null); +select to_date(cast(null as string), 'yyyy-MM-dd'); +select to_date(cast(null as string), 'invalid'); +select to_date('1986-05-23'); +select to_date('1986-05-23', null); +select to_date('1986-05-23', 'invalid'); +select to_date('1986-05-23', 'yyyy-MM-dd'); \ No newline at end of file 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 3d35732b74a5..c7f84ea383a2 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: 118 +-- Number of queries: 191 -- !query @@ -76,22 +76,6 @@ select current_date = current_date(), current_timestamp = current_timestamp() ----------------------^^^ --- !query -select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') --- !query schema -struct --- !query output -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 --- !query output -NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 - - -- !query select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15') -- !query schema @@ -1003,22 +987,626 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query -select from_unixtime(a, b) from - values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b) +select date_format(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(date '1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format(date '1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(date '1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select date_format(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select date_format(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format('1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format('1986-05-23', null) -- !query schema -struct +struct -- !query output -1970-01-01 NULL + + +-- !query +select date_format('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select date_format('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select date_format(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output NULL + + +-- !query +select date_format(timestamp '1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output NULL + + +-- !query +select date_format(timestamp '1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select date_format(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select from_unixtime(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(12345 , null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(12345 , 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1969-12-31 + + +-- !query +select unix_timestamp() +-- !query schema +struct +-- !query output +1590644388 + + +-- !query +select unix_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(date '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output NULL -- !query -select from_unixtime(12345, 'invalid') +select unix_timestamp('1986-05-23', 'invalid') -- !query schema struct<> -- !query output java.lang.IllegalArgumentException Illegal pattern character: n + + +-- !query +select unix_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(timestamp '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(date '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select to_unix_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp(cast(null as string), 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +1986-05-23 00:00:00 + + +-- !query +select to_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select to_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 00:00:00 + + +-- !query +select to_date(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date(cast(null as string), 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date('1986-05-23') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select to_date('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select to_date('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 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 46fd2a0c6790..fe17bc2033f0 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: 118 +-- Number of queries: 191 -- !query @@ -70,22 +70,6 @@ struct<(current_date() = current_date()):boolean,(current_timestamp() = current_ true true --- !query -select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') --- !query schema -struct --- !query output -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 --- !query output -NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 - - -- !query select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15') -- !query schema @@ -960,22 +944,626 @@ struct> -- !query -select from_unixtime(a, b) from - values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b) +select date_format(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(date '1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format(date '1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(date '1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select date_format(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select date_format(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format('1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format('1986-05-23', null) -- !query schema -struct +struct -- !query output -1970-01-01 NULL + + +-- !query +select date_format('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select date_format('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select date_format(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output NULL + + +-- !query +select date_format(timestamp '1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output NULL + + +-- !query +select date_format(timestamp '1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select date_format(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select from_unixtime(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(12345 , null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(12345 , 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1969-12-31 + + +-- !query +select unix_timestamp() +-- !query schema +struct +-- !query output +1590644382 + + +-- !query +select unix_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(date '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output NULL -- !query -select from_unixtime(12345, 'invalid') +select unix_timestamp('1986-05-23', 'invalid') -- !query schema struct<> -- !query output java.lang.IllegalArgumentException Illegal pattern character 'i' + + +-- !query +select unix_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(timestamp '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(date '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select to_unix_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp(cast(null as string), 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +1986-05-23 00:00:00 + + +-- !query +select to_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select to_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 00:00:00 + + +-- !query +select to_date(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date(cast(null as string), 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date('1986-05-23') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select to_date('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select to_date('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 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 edf29262b06c..98c552604733 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: 118 +-- Number of queries: 191 -- !query @@ -70,22 +70,6 @@ struct<(current_date() = current_date()):boolean,(current_timestamp() = current_ true true --- !query -select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') --- !query schema -struct --- !query output -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 --- !query output -NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 - - -- !query select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15') -- !query schema @@ -975,22 +959,626 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query -select from_unixtime(a, b) from - values (null, null), (12345, null), (null, 'invalid'), (null, 'yyyy-MM-dd'), (67890, 'yyyy-MM-dd') t(a, b) +select date_format(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(date '1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format(date '1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format(date '1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select date_format(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select date_format(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_format('1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format('1986-05-23', null) -- !query schema -struct +struct -- !query output -1970-01-01 NULL + + +-- !query +select date_format('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select date_format('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select date_format(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output NULL + + +-- !query +select date_format(timestamp '1986-05-23') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 + + +-- !query +select date_format(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output NULL + + +-- !query +select date_format(timestamp '1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select date_format(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select from_unixtime(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(12345 , null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(null , 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(12345 , 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1969-12-31 + + +-- !query +select unix_timestamp() +-- !query schema +struct +-- !query output +1590644385 + + +-- !query +select unix_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(date '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output NULL -- !query -select from_unixtime(12345, 'invalid') +select unix_timestamp('1986-05-23', 'invalid') -- !query schema struct<> -- !query output java.lang.IllegalArgumentException Illegal pattern character: n + + +-- !query +select unix_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(timestamp '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(null, 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(cast(null as date), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(date '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(date '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select to_unix_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', null) +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', 'invalid') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select to_timestamp(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp(cast(null as string), 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1986-05-23') +-- !query schema +struct +-- !query output +1986-05-23 00:00:00 + + +-- !query +select to_timestamp('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select to_timestamp('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 00:00:00 + + +-- !query +select to_date(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date(cast(null as string), 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date(cast(null as string), 'invalid') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date('1986-05-23') +-- !query schema +struct +-- !query output +1986-05-23 + + +-- !query +select to_date('1986-05-23', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date('1986-05-23', 'invalid') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select to_date('1986-05-23', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +1986-05-23 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index c12468a4e70f..5cc9e156db1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -689,8 +689,9 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { Row(secs(ts5.getTime)), Row(null))) // invalid format - checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')"), Seq( - Row(null), Row(null), Row(null), Row(null))) + val invalid = df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')") + val e = intercept[IllegalArgumentException](invalid.collect()) + assert(e.getMessage.contains('b')) } } } From 82bb96386de0bd65acfd6c1d46796666236670c3 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 28 May 2020 14:36:44 +0800 Subject: [PATCH 03/11] regen test results --- .../resources/sql-tests/inputs/datetime.sql | 4 +- .../sql-tests/results/ansi/datetime.sql.out | 37 +------------------ .../sql-tests/results/datetime-legacy.sql.out | 37 +------------------ .../sql-tests/results/datetime.sql.out | 37 +------------------ 4 files changed, 5 insertions(+), 110 deletions(-) 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 118d4bffcfbb..186b72a8bd7d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -180,7 +180,7 @@ select from_unixtime(null , 'invalid'); select from_unixtime(null , 'yyyy-MM-dd'); select from_unixtime(12345 , 'yyyy-MM-dd'); -select unix_timestamp(); +-- select unix_timestamp(); disabled because nondeterministic select unix_timestamp(null); select unix_timestamp(null, null); select unix_timestamp(null, 'yyyy-MM-dd'); @@ -235,4 +235,4 @@ select to_date(cast(null as string), 'invalid'); select to_date('1986-05-23'); select to_date('1986-05-23', null); select to_date('1986-05-23', 'invalid'); -select to_date('1986-05-23', 'yyyy-MM-dd'); \ No newline at end of file +select to_date('1986-05-23', 'yyyy-MM-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 c7f84ea383a2..f35fcea37a43 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: 191 +-- Number of queries: 187 -- !query @@ -1018,15 +1018,6 @@ struct NULL --- !query -select date_format(date '1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format(date '1986-05-23', null) -- !query schema @@ -1060,15 +1051,6 @@ struct NULL --- !query -select date_format('1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format('1986-05-23', null) -- !query schema @@ -1102,15 +1084,6 @@ struct NULL --- !query -select date_format(timestamp '1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format(timestamp '1986-05-23', null) -- !query schema @@ -1184,14 +1157,6 @@ struct 1969-12-31 --- !query -select unix_timestamp() --- !query schema -struct --- !query output -1590644388 - - -- !query select unix_timestamp(null) -- !query schema 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 fe17bc2033f0..94d5af588bb3 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: 191 +-- Number of queries: 187 -- !query @@ -975,15 +975,6 @@ struct NULL --- !query -select date_format(date '1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format(date '1986-05-23', null) -- !query schema @@ -1017,15 +1008,6 @@ struct NULL --- !query -select date_format('1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format('1986-05-23', null) -- !query schema @@ -1059,15 +1041,6 @@ struct NULL --- !query -select date_format(timestamp '1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format(timestamp '1986-05-23', null) -- !query schema @@ -1141,14 +1114,6 @@ struct 1969-12-31 --- !query -select unix_timestamp() --- !query schema -struct --- !query output -1590644382 - - -- !query select unix_timestamp(null) -- !query schema 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 98c552604733..fe9575e8e2b8 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: 191 +-- Number of queries: 187 -- !query @@ -990,15 +990,6 @@ struct NULL --- !query -select date_format(date '1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format(date '1986-05-23', null) -- !query schema @@ -1032,15 +1023,6 @@ struct NULL --- !query -select date_format('1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format('1986-05-23', null) -- !query schema @@ -1074,15 +1056,6 @@ struct NULL --- !query -select date_format(timestamp '1986-05-23') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function date_format. Expected: 2; Found: 1; line 1 pos 7 - - -- !query select date_format(timestamp '1986-05-23', null) -- !query schema @@ -1156,14 +1129,6 @@ struct 1969-12-31 --- !query -select unix_timestamp() --- !query schema -struct --- !query output -1590644385 - - -- !query select unix_timestamp(null) -- !query schema From a080397cdf50b19daa84eec726c4b586c9817493 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 28 May 2020 17:23:18 +0800 Subject: [PATCH 04/11] address comments of style stuff --- .../expressions/datetimeExpressions.scala | 35 ++++++++++++------- .../resources/sql-tests/inputs/datetime.sql | 2 +- .../sql-tests/results/ansi/datetime.sql.out | 10 +++++- .../sql-tests/results/datetime-legacy.sql.out | 10 +++++- .../sql-tests/results/datetime.sql.out | 10 +++++- 5 files changed, 50 insertions(+), 17 deletions(-) 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 8872b748210a..7e86b9c41e09 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 @@ -881,12 +881,20 @@ abstract class ToTimestamp override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - Option(left.eval(input)).map { t => + val t = left.eval(input) + if (t == null) { + null + } else { left.dataType match { - case DateType => epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor - case TimestampType => t.asInstanceOf[Long] / downScaleFactor + case DateType => + epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor + case TimestampType => + t.asInstanceOf[Long] / downScaleFactor case StringType => - Option(right.eval(input)).map { fmt => + val fmt = right.eval(input) + if (fmt == null) { + null + } else { val formatter = formatterOption.getOrElse(getFormatter(fmt.toString)) try { formatter.parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor @@ -895,9 +903,9 @@ abstract class ToTimestamp _: DateTimeException | _: ParseException => null } - }.orNull + } } - }.orNull + } } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -922,14 +930,16 @@ abstract class ToTimestamp val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val tf = TimestampFormatter.getClass.getName.stripSuffix("$") val ldf = LegacyDateFormats.getClass.getName.stripSuffix("$") + val timestampFormatter = ctx.freshName("timestampFormatter") nullSafeCodeGen(ctx, ev, (string, format) => s""" + |$tf $timestampFormatter = $tf$$.MODULE$$.apply( + | $format.toString(), + | $zid, + | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), + | true); |try { - | ${ev.value} = $tf$$.MODULE$$.apply( - | $format.toString(), - | $zid, - | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), - | true).parse($string.toString()) / $downScaleFactor; + | ${ev.value} = $timestampFormatter.parse($string.toString()) / $downScaleFactor; |} catch (java.time.format.DateTimeParseException e) { | ${ev.isNull} = true; |} catch (java.time.DateTimeException e) { @@ -1032,8 +1042,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ | $tf$$.MODULE$$.apply($format.toString(), | $zid, | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), - | false) - | .format($seconds * 1000000L)) + | false).format($seconds * 1000000L)) |""".stripMargin) } } 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 186b72a8bd7d..247eaf86d36d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -180,7 +180,7 @@ select from_unixtime(null , 'invalid'); select from_unixtime(null , 'yyyy-MM-dd'); select from_unixtime(12345 , 'yyyy-MM-dd'); --- select unix_timestamp(); disabled because nondeterministic +select unix_timestamp() = unix_timestamp(); select unix_timestamp(null); select unix_timestamp(null, null); select unix_timestamp(null, 'yyyy-MM-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 f35fcea37a43..6106839a74f0 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: 187 +-- Number of queries: 188 -- !query @@ -1157,6 +1157,14 @@ struct 1969-12-31 +-- !query +select unix_timestamp() = unix_timestamp() +-- !query schema +struct<(unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss) = unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss)):boolean> +-- !query output +true + + -- !query select unix_timestamp(null) -- !query schema 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 94d5af588bb3..e91cbbb9a924 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: 187 +-- Number of queries: 188 -- !query @@ -1114,6 +1114,14 @@ struct 1969-12-31 +-- !query +select unix_timestamp() = unix_timestamp() +-- !query schema +struct<(unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss) = unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss)):boolean> +-- !query output +true + + -- !query select unix_timestamp(null) -- !query schema 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 fe9575e8e2b8..e8d7a616ce05 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: 187 +-- Number of queries: 188 -- !query @@ -1129,6 +1129,14 @@ struct 1969-12-31 +-- !query +select unix_timestamp() = unix_timestamp() +-- !query schema +struct<(unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss) = unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss)):boolean> +-- !query output +true + + -- !query select unix_timestamp(null) -- !query schema From 3c68c4e3f09c47469c364ad24778fcaab4e94c7b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 28 May 2020 17:24:01 +0800 Subject: [PATCH 05/11] import --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 2 -- 1 file changed, 2 deletions(-) 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 7e86b9c41e09..191c139e4ce9 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 @@ -23,8 +23,6 @@ import java.time.format.DateTimeParseException import java.time.temporal.IsoFields import java.util.Locale -import scala.util.control.NonFatal - import org.apache.commons.text.StringEscapeUtils import org.apache.spark.sql.AnalysisException From ce2eff0f3edbf34c2f2a92627486898bb1d65380 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 28 May 2020 17:36:07 +0800 Subject: [PATCH 06/11] mv validatePatternString function --- .../sql/catalyst/expressions/datetimeExpressions.scala | 4 +--- .../spark/sql/catalyst/util/TimestampFormatter.scala | 8 ++++---- 2 files changed, 5 insertions(+), 7 deletions(-) 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 191c139e4ce9..f3ec6320c1c2 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 @@ -66,13 +66,11 @@ trait TimestampFormatterHelper extends TimeZoneAwareExpression { } else None final protected def getFormatter(fmt: String): TimestampFormatter = { - val formatter = TimestampFormatter( + TimestampFormatter( format = fmt, zoneId = zoneId, legacyFormat = SIMPLE_DATE_FORMAT, needVarLengthSecondFraction = isParsing) - formatter.validatePatternString() - formatter } } 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 8428964d4570..1d1ecb58742b 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 @@ -284,14 +284,14 @@ object TimestampFormatter { legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT, needVarLengthSecondFraction: Boolean = false): TimestampFormatter = { val pattern = format.getOrElse(defaultPattern) - if (SQLConf.get.legacyTimeParserPolicy == LEGACY) { + val formatter = if (SQLConf.get.legacyTimeParserPolicy == LEGACY) { getLegacyFormatter(pattern, zoneId, locale, legacyFormat) } else { - val tf = new Iso8601TimestampFormatter( + new Iso8601TimestampFormatter( pattern, zoneId, locale, legacyFormat, needVarLengthSecondFraction) - tf.validatePatternString() - tf } + formatter.validatePatternString() + formatter } def getLegacyFormatter( From 0680855705a6a19d2a60b746128085a3a320501d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 29 May 2020 00:30:04 +0800 Subject: [PATCH 07/11] improve test coverage --- .../resources/sql-tests/inputs/datetime.sql | 23 +++ .../sql-tests/results/ansi/datetime.sql.out | 140 +++++++++++++++++- .../sql-tests/results/datetime-legacy.sql.out | 138 ++++++++++++++++- .../sql-tests/results/datetime.sql.out | 140 +++++++++++++++++- 4 files changed, 438 insertions(+), 3 deletions(-) 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 247eaf86d36d..4275e71658f2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -157,6 +157,15 @@ select from_json('{"date":"26/October/2015"}', 'date Date', map('dateFormat', 'd 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')); +create temporary view dnf as select d, f from values + ('1986-05-23', 'yyyy-MM-dd'), + ('1986-05-23a', 'yyyy-MM-dd'), + ('1986-05-23', 'invalid'), + ('1986-05-23', null), + (null, 'yyyy-MM-dd'), + (null, 'invalid'), + (null, null) t(d, f); + select date_format(null, null); select date_format(null, 'yyyy-MM-dd'); select date_format(null, 'invalid'); @@ -172,6 +181,8 @@ select date_format(cast(null as timestamp ), 'yyyy-MM-dd'); select date_format(timestamp '1986-05-23', null); select date_format(timestamp '1986-05-23', 'invalid'); select date_format(timestamp '1986-05-23', 'yyyy-MM-dd'); +select date_format(d, f) from dnf where f != 'invalid' or d is null or f is null ; +select date_format(d, f) from dnf where f = 'invalid'; select from_unixtime(null); select from_unixtime(null , null); @@ -179,6 +190,10 @@ select from_unixtime(12345 , null); select from_unixtime(null , 'invalid'); select from_unixtime(null , 'yyyy-MM-dd'); select from_unixtime(12345 , 'yyyy-MM-dd'); +select from_unixtime(12345, f) from dnf where f <> 'invalid'; +select from_unixtime(9223372036854775807L, f) from dnf where f <> 'invalid'; +select from_unixtime(12345, f) from dnf where f = 'invalid'; +select from_unixtime(null, f) from dnf; select unix_timestamp() = unix_timestamp(); select unix_timestamp(null); @@ -195,11 +210,19 @@ select unix_timestamp('1986-05-23'); select unix_timestamp('1986-05-23', null); select unix_timestamp('1986-05-23', 'invalid'); select unix_timestamp('1986-05-23', 'yyyy-MM-dd'); +select unix_timestamp('1986-05-23a', 'yyyy-MM-dd'); select unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd'); select unix_timestamp(timestamp '1986-05-23'); select unix_timestamp(timestamp '1986-05-23', null); select unix_timestamp(timestamp '1986-05-23', 'invalid'); select unix_timestamp(timestamp '1986-05-23', 'yyyy-MM-dd'); +-- these test un-foldable codegen version for [[ToTimestamp]] interface, covers all of its implementations +-- which are unix_timestamp, to_unix_timestamp, to_timestamp and to_date +select unix_timestamp(d, f) from dnf where d = '1986-05-23a'; +select unix_timestamp(d, f) from dnf where d != '1986-05-23a' and (f != 'invalid' or d is null or f is null); +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is null; +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is not null; +select unix_timestamp(cast(d as date), f), unix_timestamp(cast(d as timestamp), f) from dnf; select to_unix_timestamp(null); select to_unix_timestamp(null, null); 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 6106839a74f0..2b7c03581c34 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: 188 +-- Number of queries: 201 -- !query @@ -986,6 +986,21 @@ 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 +create temporary view dnf as select d, f from values + ('1986-05-23', 'yyyy-MM-dd'), + ('1986-05-23a', 'yyyy-MM-dd'), + ('1986-05-23', 'invalid'), + ('1986-05-23', null), + (null, 'yyyy-MM-dd'), + (null, 'invalid'), + (null, null) t(d, f) +-- !query schema +struct<> +-- !query output + + + -- !query select date_format(null, null) -- !query schema @@ -1109,6 +1124,28 @@ struct 1986-05-23 +-- !query +select date_format(d, f) from dnf where f != 'invalid' or d is null or f is null +-- !query schema +struct +-- !query output +1986-05-23 +NULL +NULL +NULL +NULL +NULL + + +-- !query +select date_format(d, f) from dnf where f = 'invalid' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + -- !query select from_unixtime(null) -- !query schema @@ -1157,6 +1194,49 @@ struct 1969-12-31 +-- !query +select from_unixtime(12345, f) from dnf where f <> 'invalid' +-- !query schema +struct +-- !query output +1969-12-31 +1969-12-31 +1969-12-31 + + +-- !query +select from_unixtime(9223372036854775807L, f) from dnf where f <> 'invalid' +-- !query schema +struct +-- !query output +1969-12-31 +1969-12-31 +1969-12-31 + + +-- !query +select from_unixtime(12345, f) from dnf where f = 'invalid' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select from_unixtime(null, f) from dnf +-- !query schema +struct +-- !query output +NULL +NULL +NULL +NULL +NULL +NULL +NULL + + -- !query select unix_timestamp() = unix_timestamp() -- !query schema @@ -1278,6 +1358,15 @@ struct 517215600 +-- !query +select unix_timestamp('1986-05-23a', 'yyyy-MM-dd') +-- !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 '1986-05-23a' 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 unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') -- !query schema @@ -1318,6 +1407,55 @@ struct 517215600 +-- !query +select unix_timestamp(d, f) from dnf where d = '1986-05-23a' +-- !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 '1986-05-23a' 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 unix_timestamp(d, f) from dnf where d != '1986-05-23a' and (f != 'invalid' or d is null or f is null) +-- !query schema +struct +-- !query output +517215600 +NULL + + +-- !query +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is not null +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select unix_timestamp(cast(d as date), f), unix_timestamp(cast(d as timestamp), f) from dnf +-- !query schema +struct +-- !query output +517215600 517215600 +517215600 517215600 +517215600 517215600 +NULL NULL +NULL NULL +NULL NULL +NULL NULL + + -- !query select to_unix_timestamp(null) -- !query schema 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 e91cbbb9a924..761559068ab0 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: 188 +-- Number of queries: 201 -- !query @@ -943,6 +943,21 @@ struct> {"date":2015-10-26} +-- !query +create temporary view dnf as select d, f from values + ('1986-05-23', 'yyyy-MM-dd'), + ('1986-05-23a', 'yyyy-MM-dd'), + ('1986-05-23', 'invalid'), + ('1986-05-23', null), + (null, 'yyyy-MM-dd'), + (null, 'invalid'), + (null, null) t(d, f) +-- !query schema +struct<> +-- !query output + + + -- !query select date_format(null, null) -- !query schema @@ -1066,6 +1081,28 @@ struct 1986-05-23 +-- !query +select date_format(d, f) from dnf where f != 'invalid' or d is null or f is null +-- !query schema +struct +-- !query output +1986-05-23 +NULL +NULL +NULL +NULL +NULL + + +-- !query +select date_format(d, f) from dnf where f = 'invalid' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + -- !query select from_unixtime(null) -- !query schema @@ -1114,6 +1151,49 @@ struct 1969-12-31 +-- !query +select from_unixtime(12345, f) from dnf where f <> 'invalid' +-- !query schema +struct +-- !query output +1969-12-31 +1969-12-31 +1969-12-31 + + +-- !query +select from_unixtime(9223372036854775807L, f) from dnf where f <> 'invalid' +-- !query schema +struct +-- !query output +1969-12-31 +1969-12-31 +1969-12-31 + + +-- !query +select from_unixtime(12345, f) from dnf where f = 'invalid' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select from_unixtime(null, f) from dnf +-- !query schema +struct +-- !query output +NULL +NULL +NULL +NULL +NULL +NULL +NULL + + -- !query select unix_timestamp() = unix_timestamp() -- !query schema @@ -1235,6 +1315,14 @@ struct 517215600 +-- !query +select unix_timestamp('1986-05-23a', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +517215600 + + -- !query select unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') -- !query schema @@ -1275,6 +1363,54 @@ struct 517215600 +-- !query +select unix_timestamp(d, f) from dnf where d = '1986-05-23a' +-- !query schema +struct +-- !query output +517215600 + + +-- !query +select unix_timestamp(d, f) from dnf where d != '1986-05-23a' and (f != 'invalid' or d is null or f is null) +-- !query schema +struct +-- !query output +517215600 +NULL + + +-- !query +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is not null +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'i' + + +-- !query +select unix_timestamp(cast(d as date), f), unix_timestamp(cast(d as timestamp), f) from dnf +-- !query schema +struct +-- !query output +517215600 517215600 +517215600 517215600 +517215600 517215600 +NULL NULL +NULL NULL +NULL NULL +NULL NULL + + -- !query select to_unix_timestamp(null) -- !query schema 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 e8d7a616ce05..2285ef3d8649 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: 188 +-- Number of queries: 201 -- !query @@ -958,6 +958,21 @@ 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 +create temporary view dnf as select d, f from values + ('1986-05-23', 'yyyy-MM-dd'), + ('1986-05-23a', 'yyyy-MM-dd'), + ('1986-05-23', 'invalid'), + ('1986-05-23', null), + (null, 'yyyy-MM-dd'), + (null, 'invalid'), + (null, null) t(d, f) +-- !query schema +struct<> +-- !query output + + + -- !query select date_format(null, null) -- !query schema @@ -1081,6 +1096,28 @@ struct 1986-05-23 +-- !query +select date_format(d, f) from dnf where f != 'invalid' or d is null or f is null +-- !query schema +struct +-- !query output +1986-05-23 +NULL +NULL +NULL +NULL +NULL + + +-- !query +select date_format(d, f) from dnf where f = 'invalid' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + -- !query select from_unixtime(null) -- !query schema @@ -1129,6 +1166,49 @@ struct 1969-12-31 +-- !query +select from_unixtime(12345, f) from dnf where f <> 'invalid' +-- !query schema +struct +-- !query output +1969-12-31 +1969-12-31 +1969-12-31 + + +-- !query +select from_unixtime(9223372036854775807L, f) from dnf where f <> 'invalid' +-- !query schema +struct +-- !query output +1969-12-31 +1969-12-31 +1969-12-31 + + +-- !query +select from_unixtime(12345, f) from dnf where f = 'invalid' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select from_unixtime(null, f) from dnf +-- !query schema +struct +-- !query output +NULL +NULL +NULL +NULL +NULL +NULL +NULL + + -- !query select unix_timestamp() = unix_timestamp() -- !query schema @@ -1250,6 +1330,15 @@ struct 517215600 +-- !query +select unix_timestamp('1986-05-23a', 'yyyy-MM-dd') +-- !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 '1986-05-23a' 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 unix_timestamp(cast(null as timestamp ), 'yyyy-MM-dd') -- !query schema @@ -1290,6 +1379,55 @@ struct 517215600 +-- !query +select unix_timestamp(d, f) from dnf where d = '1986-05-23a' +-- !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 '1986-05-23a' 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 unix_timestamp(d, f) from dnf where d != '1986-05-23a' and (f != 'invalid' or d is null or f is null) +-- !query schema +struct +-- !query output +517215600 +NULL + + +-- !query +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(d, f) from dnf where f = 'invalid' and d is not null +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: n + + +-- !query +select unix_timestamp(cast(d as date), f), unix_timestamp(cast(d as timestamp), f) from dnf +-- !query schema +struct +-- !query output +517215600 517215600 +517215600 517215600 +517215600 517215600 +NULL NULL +NULL NULL +NULL NULL +NULL NULL + + -- !query select to_unix_timestamp(null) -- !query schema From 2e940bd365242561ed5e110e2adbe7f51bc66b49 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 9 Jun 2020 14:35:19 +0800 Subject: [PATCH 08/11] tests --- .../expressions/DateExpressionsSuite.scala | 48 +++++++------------ 1 file changed, 18 insertions(+), 30 deletions(-) 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 2dc5990eb610..c308c5867b4d 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 @@ -23,6 +23,8 @@ import java.time.{Instant, LocalDate, ZoneId} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit._ +import scala.reflect.ClassTag + import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection @@ -777,8 +779,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), null) - checkEvaluation( - FromUnixTime(Literal(0L), Literal("not a valid format"), timeZoneId), null) // SPARK-28072 The codegen path for non-literal input should also work checkEvaluation( @@ -792,7 +792,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } // Test escaping of format - GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\"quote"), UTC_OPT) :: Nil) + GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\""), UTC_OPT) :: Nil) } test("unix_timestamp") { @@ -854,15 +854,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), MICROSECONDS.toSeconds( DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) - checkEvaluation( - UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) } } } } // Test escaping of format GenerateUnsafeProjection.generate( - UnixTimestamp(Literal("2015-07-24"), Literal("\"quote"), UTC_OPT) :: Nil) + UnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) } test("to_unix_timestamp") { @@ -920,10 +918,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { Literal(date1), Literal.create(null, StringType), timeZoneId), MICROSECONDS.toSeconds( DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) - checkEvaluation( - ToUnixTimestamp( - Literal("2015-07-24"), - Literal("not a valid format"), timeZoneId), null) // SPARK-28072 The codegen path for non-literal input should also work checkEvaluation( @@ -940,7 +934,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } // Test escaping of format GenerateUnsafeProjection.generate( - ToUnixTimestamp(Literal("2015-07-24"), Literal("\"quote"), UTC_OPT) :: Nil) + ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) } test("datediff") { @@ -1171,34 +1165,28 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Disable week-based date fields and quarter fields for parsing") { - def checkSparkUpgrade(c: Char): Unit = { - checkExceptionInExpression[SparkUpgradeException]( - new ParseToTimestamp(Literal("1"), Literal(c.toString)).child, "3.0") - checkExceptionInExpression[SparkUpgradeException]( - new ParseToDate(Literal("1"), Literal(c.toString)).child, "3.0") - checkExceptionInExpression[SparkUpgradeException]( - ToUnixTimestamp(Literal("1"), Literal(c.toString)), "3.0") - checkExceptionInExpression[SparkUpgradeException]( - UnixTimestamp(Literal("1"), Literal(c.toString)), "3.0") + def checkException[T <: Exception : ClassTag](c: String, onlyParsing: Boolean = false): Unit = { + checkExceptionInExpression[T](new ParseToTimestamp(Literal("1"), Literal(c)).child, c) + checkExceptionInExpression[T](new ParseToDate(Literal("1"), Literal(c)).child, c) + checkExceptionInExpression[T](ToUnixTimestamp(Literal("1"), Literal(c)), c) + checkExceptionInExpression[T](UnixTimestamp(Literal("1"), Literal(c)), c) + if (!onlyParsing) { + checkExceptionInExpression[T](DateFormatClass(CurrentTimestamp(), Literal(c)), c) + checkExceptionInExpression[T](FromUnixTime(Literal(0L), Literal(c)), c) + } } - def checkNullify(c: Char): Unit = { - checkEvaluation(new ParseToTimestamp(Literal("1"), Literal(c.toString)).child, null) - checkEvaluation(new ParseToDate(Literal("1"), Literal(c.toString)).child, null) - checkEvaluation(ToUnixTimestamp(Literal("1"), Literal(c.toString)), null) - checkEvaluation(UnixTimestamp(Literal("1"), Literal(c.toString)), null) - } + val unsupportedLettersForParsing = Set('E', 'F', 'q', 'Q') Seq('Y', 'W', 'w', 'E', 'u', 'F').foreach { l => - checkSparkUpgrade(l) + checkException[SparkUpgradeException](l.toString, unsupportedLettersForParsing.contains(l)) } - Seq('q', 'Q').foreach { l => - checkNullify(l) + Seq('q', 'Q', 'e', 'c', 'A', 'n', 'N', 'p').foreach { l => + checkException[IllegalArgumentException](l.toString, unsupportedLettersForParsing.contains(l)) } } - test("SPARK-31896: Handle am-pm timestamp parsing when hour is missing") { checkEvaluation( new ParseToTimestamp(Literal("PM"), Literal("a")).child, From daac8dcd391fd36d1b6bd486e0e9324250974c0d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 9 Jun 2020 15:13:54 +0800 Subject: [PATCH 09/11] rm tests --- .../resources/sql-tests/inputs/datetime.sql | 14 --- .../sql-tests/results/ansi/datetime.sql.out | 118 +----------------- .../sql-tests/results/datetime-legacy.sql.out | 109 +--------------- .../sql-tests/results/datetime.sql.out | 118 +----------------- 4 files changed, 3 insertions(+), 356 deletions(-) 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 a63bb8526da4..06765627f554 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -138,25 +138,11 @@ select to_timestamp("2019 40", "yyyy mm"); select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss"); -- Unsupported narrow text style -select date_format(date '2020-05-23', 'GGGGG'); -select date_format(date '2020-05-23', 'MMMMM'); -select date_format(date '2020-05-23', 'LLLLL'); -select date_format(timestamp '2020-05-23', 'EEEEE'); -select date_format(timestamp '2020-05-23', 'uuuuu'); -select date_format('2020-05-23', 'QQQQQ'); -select date_format('2020-05-23', 'qqqqq'); select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG'); select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE'); select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE'); select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE'); -select from_unixtime(12345, 'MMMMM'); -select from_unixtime(54321, 'QQQQQ'); -select from_unixtime(23456, 'aaaaa'); select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); 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')); - -select from_unixtime(1, 'yyyyyyyyyyy-MM-dd'); -select date_format(timestamp '2018-11-17 13:33:33', 'yyyyyyyyyy-MM-dd HH:mm:ss'); -select date_format(date '2018-11-17', 'yyyyyyyyyyy-MM-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 36156da84ee8..26adb40ce1b1 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: 103 -- !query @@ -814,69 +814,6 @@ struct 2019-01-01 10:10:10 --- !query -select date_format(date '2020-05-23', '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 '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 - - --- !query -select date_format(date '2020-05-23', 'MMMMM') --- !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 'MMMMM' 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 date_format(date '2020-05-23', 'LLLLL') --- !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 'LLLLL' 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 date_format(timestamp '2020-05-23', '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 '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 - - --- !query -select date_format(timestamp '2020-05-23', 'uuuuu') --- !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 'uuuuu' 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 date_format('2020-05-23', 'QQQQQ') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Too many pattern letters: Q - - --- !query -select date_format('2020-05-23', 'qqqqq') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Too many pattern letters: q - - -- !query select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !query schema @@ -913,33 +850,6 @@ 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 --- !query -select from_unixtime(12345, 'MMMMM') --- !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 'MMMMM' 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 from_unixtime(54321, 'QQQQQ') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Too many pattern letters: Q - - --- !query -select from_unixtime(23456, 'aaaaa') --- !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 'aaaaa' 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 from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) -- !query schema @@ -974,29 +884,3 @@ 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 from_unixtime(1, 'yyyyyyyyyyy-MM-dd') --- !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 'yyyyyyyyyyy-MM-dd' 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 date_format(timestamp '2018-11-17 13:33:33', 'yyyyyyyyyy-MM-dd HH:mm:ss') --- !query schema -struct --- !query output -0000002018-11-17 13:33:33 - - --- !query -select date_format(date '2018-11-17', 'yyyyyyyyyyy-MM-dd') --- !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 'yyyyyyyyyyy-MM-dd' 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 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 c72b77a76211..15092f0a27c1 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: 103 -- !query @@ -786,64 +786,6 @@ struct 2019-01-01 10:10:10 --- !query -select date_format(date '2020-05-23', 'GGGGG') --- !query schema -struct --- !query output -AD - - --- !query -select date_format(date '2020-05-23', 'MMMMM') --- !query schema -struct --- !query output -May - - --- !query -select date_format(date '2020-05-23', 'LLLLL') --- !query schema -struct --- !query output -May - - --- !query -select date_format(timestamp '2020-05-23', 'EEEEE') --- !query schema -struct --- !query output -Saturday - - --- !query -select date_format(timestamp '2020-05-23', 'uuuuu') --- !query schema -struct --- !query output -00006 - - --- !query -select date_format('2020-05-23', 'QQQQQ') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Illegal pattern character 'Q' - - --- !query -select date_format('2020-05-23', 'qqqqq') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Illegal pattern character 'q' - - -- !query select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !query schema @@ -876,31 +818,6 @@ struct 1590130800 --- !query -select from_unixtime(12345, 'MMMMM') --- !query schema -struct --- !query output -December - - --- !query -select from_unixtime(54321, 'QQQQQ') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Illegal pattern character 'Q' - - --- !query -select from_unixtime(23456, 'aaaaa') --- !query schema -struct --- !query output -PM - - -- !query select from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) -- !query schema @@ -931,27 +848,3 @@ select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy struct> -- !query output {"date":2015-10-26} - - --- !query -select from_unixtime(1, 'yyyyyyyyyyy-MM-dd') --- !query schema -struct --- !query output -00000001969-12-31 - - --- !query -select date_format(timestamp '2018-11-17 13:33:33', 'yyyyyyyyyy-MM-dd HH:mm:ss') --- !query schema -struct --- !query output -0000002018-11-17 13:33:33 - - --- !query -select date_format(date '2018-11-17', 'yyyyyyyyyyy-MM-dd') --- !query schema -struct --- !query output -00000002018-11-17 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 8a51c9c617a3..b80f36e9c234 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: 103 -- !query @@ -786,69 +786,6 @@ struct 2019-01-01 10:10:10 --- !query -select date_format(date '2020-05-23', '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 '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 - - --- !query -select date_format(date '2020-05-23', 'MMMMM') --- !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 'MMMMM' 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 date_format(date '2020-05-23', 'LLLLL') --- !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 'LLLLL' 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 date_format(timestamp '2020-05-23', '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 '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 - - --- !query -select date_format(timestamp '2020-05-23', 'uuuuu') --- !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 'uuuuu' 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 date_format('2020-05-23', 'QQQQQ') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Too many pattern letters: Q - - --- !query -select date_format('2020-05-23', 'qqqqq') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Too many pattern letters: q - - -- !query select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !query schema @@ -885,33 +822,6 @@ 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 --- !query -select from_unixtime(12345, 'MMMMM') --- !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 'MMMMM' 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 from_unixtime(54321, 'QQQQQ') --- !query schema -struct<> --- !query output -java.lang.IllegalArgumentException -Too many pattern letters: Q - - --- !query -select from_unixtime(23456, 'aaaaa') --- !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 'aaaaa' 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 from_json('{"time":"26/October/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) -- !query schema @@ -946,29 +856,3 @@ 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 from_unixtime(1, 'yyyyyyyyyyy-MM-dd') --- !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 'yyyyyyyyyyy-MM-dd' 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 date_format(timestamp '2018-11-17 13:33:33', 'yyyyyyyyyy-MM-dd HH:mm:ss') --- !query schema -struct --- !query output -0000002018-11-17 13:33:33 - - --- !query -select date_format(date '2018-11-17', 'yyyyyyyyyyy-MM-dd') --- !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 'yyyyyyyyyyy-MM-dd' 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 From 9d578f1316ca00b6aee2254fd42b7379372e5a86 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 9 Jun 2020 17:43:05 +0800 Subject: [PATCH 10/11] address comments --- .../catalyst/expressions/DateExpressionsSuite.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) 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 c308c5867b4d..f248a3454f39 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 @@ -1163,27 +1163,25 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { MillisToTimestamp(Literal(-92233720368547758L)), "long overflow") } - test("Disable week-based date fields and quarter fields for parsing") { + test("Consistent error handling for datetime formatting and parsing functions") { - def checkException[T <: Exception : ClassTag](c: String, onlyParsing: Boolean = false): Unit = { + def checkException[T <: Exception : ClassTag](c: String): Unit = { checkExceptionInExpression[T](new ParseToTimestamp(Literal("1"), Literal(c)).child, c) checkExceptionInExpression[T](new ParseToDate(Literal("1"), Literal(c)).child, c) checkExceptionInExpression[T](ToUnixTimestamp(Literal("1"), Literal(c)), c) checkExceptionInExpression[T](UnixTimestamp(Literal("1"), Literal(c)), c) - if (!onlyParsing) { + if (!Set("E", "F", "q", "Q").contains(c)) { checkExceptionInExpression[T](DateFormatClass(CurrentTimestamp(), Literal(c)), c) checkExceptionInExpression[T](FromUnixTime(Literal(0L), Literal(c)), c) } } - val unsupportedLettersForParsing = Set('E', 'F', 'q', 'Q') - Seq('Y', 'W', 'w', 'E', 'u', 'F').foreach { l => - checkException[SparkUpgradeException](l.toString, unsupportedLettersForParsing.contains(l)) + checkException[SparkUpgradeException](l.toString) } Seq('q', 'Q', 'e', 'c', 'A', 'n', 'N', 'p').foreach { l => - checkException[IllegalArgumentException](l.toString, unsupportedLettersForParsing.contains(l)) + checkException[IllegalArgumentException](l.toString) } } From 8139dfc1d064ab7b1b1d7be1af0c8f903d3e6b13 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 9 Jun 2020 18:31:42 +0800 Subject: [PATCH 11/11] fix ut --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 8580a41670ff..c5cf447c103b 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 @@ -1009,7 +1009,8 @@ abstract class UnixTime extends ToTimestamp { since = "1.5.0") // scalastyle:on line.size.limit case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimestampFormatterHelper with ImplicitCastInputTypes { + extends BinaryExpression with TimestampFormatterHelper with ImplicitCastInputTypes + with NullIntolerant { def this(sec: Expression, format: Expression) = this(sec, format, None)