diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 76c3b1d80b294..2baf71153d748 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -674,7 +674,7 @@ object FunctionRegistry { expression[WindowTime]("window_time"), expression[MakeDate]("make_date"), expression[MakeTime]("make_time"), - expression[MakeTimestamp]("make_timestamp"), + expressionBuilder("make_timestamp", MakeTimestampExpressionBuilder), expression[TryMakeTimestamp]("try_make_timestamp"), expression[MonthName]("monthname"), // We keep the 2 expression builders below to have different function docs. 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 2f937f5fbf46d..299802beea8d2 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 @@ -2791,6 +2791,10 @@ object TryMakeTimestampLTZExpressionBuilder extends ExpressionBuilder { } } +/** + * This make_timestamp overload creates a timestamp from: + * - a year, month, day, hour, min, sec and optional timezone. + */ // scalastyle:off line.size.limit @ExpressionDescription( usage = "_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create timestamp from year, month, day, hour, min, sec and timezone fields. The result data type is consistent with the value of configuration `spark.sql.timestampType`. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.", @@ -3077,6 +3081,151 @@ case class TryMakeTimestamp( } } +/** + * This make_timestamp overload creates a timestamp from: + * - a date, optional time, and optional timezone. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(date [, time [, timezone]]) - Create timestamp from date and time fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead. + """, + arguments = """ + Arguments: + * date - a date expression + * time - a time expression + * timezone - the time zone identifier. For example, CET, UTC and etc. + """, + examples = """ + Examples: + > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887'); + 2014-12-27 22:30:45.887 + > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887', 'CET'); + 2014-12-27 21:30:45.887 + """, + group = "datetime_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +case class MakeTimestampFromDateTime( + date: Expression, + time: Expression, + timezone: Option[Expression] = None) + extends Expression with RuntimeReplaceable with ExpectsInputTypes { + + def this(date: Expression, time: Expression) = + this(date, time, None) + def this(date: Expression, time: Expression, timezone: Expression) = + this(date, time, Some(timezone)) + + override def children: Seq[Expression] = Seq(date, time) ++ timezone + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType, AnyTimeType) ++ + timezone.map(_ => StringTypeWithCollation(supportsTrimCollation = true)) + + override def replacement: Expression = { + // If timezone is not provided, we use UTC, i.e. +00:00. + val zoneIdExpr = timezone.getOrElse(Literal("+00:00")) + StaticInvoke( + classOf[DateTimeUtils.type], + TimestampType, + "makeTimestamp", + Seq(date, time, zoneIdExpr), + Seq(date.dataType, time.dataType, zoneIdExpr.dataType), + returnNullable = children.exists(_.nullable)) + } + + override def prettyName: String = "make_timestamp" + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): Expression = { + val timezoneOpt = if (timezone.isDefined) Some(newChildren(2)) else None + copy(date = newChildren(0), time = newChildren(1), timezone = timezoneOpt) + } +} + +/** + * This make_timestamp expression builder creates a timestamp from: + * - a year, month, day, hour, min, sec and optional timezone, or + * - a date, optional time, and optional timezone. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(year, month, day, hour, min, sec[, timezone]) - Create the current timestamp with local time zone from year, month, day, hour, min, sec and timezone fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead. + + _FUNC_(date [, time [, timezone]]) - Create timestamp from date and time fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead. + """, + arguments = """ + Arguments: + * year - the year to represent, from 1 to 9999 + * month - the month-of-year to represent, from 1 (January) to 12 (December) + * day - the day-of-month to represent, from 1 to 31 + * hour - the hour-of-day to represent, from 0 to 23 + * min - the minute-of-hour to represent, from 0 to 59 + * sec - the second-of-minute and its micro-fraction to represent, from + 0 to 60. If the sec argument equals to 60, the seconds field is set + to 0 and 1 minute is added to the final timestamp. + * date - a date expression + * time - a time expression + * timezone - the time zone identifier. For example, CET, UTC and etc. + """, + examples = """ + Examples: + > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887); + 2014-12-28 06:30:45.887 + > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET'); + 2014-12-27 21:30:45.887 + > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887'); + 2014-12-27 22:30:45.887 + > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887', 'CET'); + 2014-12-27 21:30:45.887 + """, + group = "datetime_funcs", + since = "4.1.0") +// scalastyle:on line.size.limit +object MakeTimestampExpressionBuilder extends ExpressionBuilder { + override def build(funcName: String, expressions: Seq[Expression]): Expression = { + val numArgs = expressions.length + if (numArgs == 2) { + // date, time + MakeTimestampFromDateTime( + expressions(0), + expressions(1) + ) + } else if (numArgs == 3) { + // date, time, timezone + MakeTimestampFromDateTime( + expressions(0), + expressions(1), + Some(expressions(2)) + ) + } else if (numArgs == 6) { + // year, month, day, hour, min, sec + MakeTimestamp( + expressions(0), + expressions(1), + expressions(2), + expressions(3), + expressions(4), + expressions(5) + ) + } else if (numArgs == 7) { + // year, month, day, hour, min, sec, timezone + MakeTimestamp( + expressions(0), + expressions(1), + expressions(2), + expressions(3), + expressions(4), + expressions(5), + Some(expressions(6)) + ) + } else { + throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(2, 3, 6, 7), numArgs) + } + } +} + object DatePart { def parseExtractField( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 5037f304d1846..e60dc3976ed9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -848,6 +848,34 @@ object DateTimeUtils extends SparkDateTimeUtils { localDateTimeToMicros(LocalDateTime.of(daysToLocalDate(days), nanosToLocalTime(nanos))) } + /** + * Makes a timestamp from a date and a local time. + * + * @param days The number of days since the epoch. 1970-01-01. + * Negative numbers represent earlier days. + * @param nanos The number of nanoseconds within the day since the midnight. + * @param zoneId The time zone ID at which the operation is performed. + * @return The number of microseconds since the epoch of 1970-01-01 00:00:00Z. + */ + def makeTimestamp(days: Int, nanos: Long, zoneId: ZoneId): Long = { + val ldt = LocalDateTime.of(daysToLocalDate(days), nanosToLocalTime(nanos)) + instantToMicros(ldt.atZone(zoneId).toInstant) + } + + /** + * Makes a timestamp from a date and a local time with timezone string. + * + * @param days The number of days since the epoch. 1970-01-01. + * Negative numbers represent earlier days. + * @param nanos The number of nanoseconds within the day since the midnight. + * @param timezone The time zone string. + * @return The number of microseconds since the epoch of 1970-01-01 00:00:00Z. + */ + def makeTimestamp(days: Int, nanos: Long, timezone: UTF8String): Long = { + val zoneId = getZoneId(timezone.toString) + makeTimestamp(days, nanos, zoneId) + } + /** * Adds a day-time interval to a time. * 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 02edd36952702..2160e94b19978 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 @@ -2142,14 +2142,132 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("make timestamp_ntz from date and time") { - def dateLit(d: String): Expression = Literal(LocalDate.parse(d)) - def timeLit(t: String): Expression = Literal(LocalTime.parse(t)) - def tsNtz(s: String): Long = localDateTimeToMicros(LocalDateTime.parse(s), UTC) + /** + * Helper method to create a DATE literal from a string in date format. + */ + private def dateLit(date: String): Expression = Literal(LocalDate.parse(date)) + + /** + * Helper method to create a TIME literal from a string in time format. + */ + private def timeLit(time: String): Expression = Literal(LocalTime.parse(time)) + + /** + * Helper method to get the microseconds from a timestamp represented as a string. + */ + private def timestampToMicros(timestamp: String, zoneId: ZoneId): Long = { + localDateTimeToMicros(LocalDateTime.parse(timestamp), zoneId) + } + + test("SPARK-51415: make timestamp from date and time") { + // Test with valid date and time. + checkEvaluation( + MakeTimestampFromDateTime(dateLit("2023-10-01"), timeLit("12:34:56.123456")), + timestampToMicros("2023-10-01T12:34:56.123456", UTC) + ) + // Test with null date. + checkEvaluation( + MakeTimestampFromDateTime(Literal(null, DateType), timeLit("12:34:56.123456")), + null + ) + // Test with null time. + checkEvaluation( + MakeTimestampFromDateTime(dateLit("2023-10-01"), Literal(null, TimeType())), + null + ) + // Test with null date and null time. + checkEvaluation( + MakeTimestampFromDateTime(Literal(null, DateType), Literal(null, TimeType())), + null + ) + } + + test("SPARK-51415: make timestamp from date, time, and timezone") { + // Test with valid date, time, and timezone. + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + timeLit("12:34:56.123456"), + Some(Literal("-09:30")) + ), + timestampToMicros("2023-10-01T12:34:56.123456", MIT) + ) + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + timeLit("12:34:56.123456"), + Some(Literal("-08:00")) + ), + timestampToMicros("2023-10-01T12:34:56.123456", PST) + ) + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + timeLit("12:34:56.123456"), + Some(Literal("+00:00")) + ), + timestampToMicros("2023-10-01T12:34:56.123456", UTC) + ) + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + timeLit("12:34:56.123456"), + Some(Literal("+01:00")) + ), + timestampToMicros("2023-10-01T12:34:56.123456", CET) + ) + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + timeLit("12:34:56.123456"), + Some(Literal("+09:00")) + ), + timestampToMicros("2023-10-01T12:34:56.123456", JST) + ) + + // Test with null date. + checkEvaluation( + MakeTimestampFromDateTime( + Literal(null, DateType), + timeLit("12:34:56.123456"), + Some(Literal("+00:00")) + ), + null + ) + // Test with null time. + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + Literal(null, TimeType()), + Some(Literal("+00:00")) + ), + null + ) + // Test with null timezone. + checkEvaluation( + MakeTimestampFromDateTime( + dateLit("2023-10-01"), + timeLit("12:34:56.123456"), + Some(Literal(null, StringType)) + ), + null + ) + // Test with null date and null time. + checkEvaluation( + MakeTimestampFromDateTime( + Literal(null, DateType), + Literal(null, TimeType()), + Some(Literal("+00:00")) + ), + null + ) + } + + test("make timestamp_ntz from date and time") { checkEvaluation(MakeTimestampNTZ(dateLit("1970-01-01"), timeLit("00:00:00")), 0L) checkEvaluation(MakeTimestampNTZ(dateLit("2025-06-20"), timeLit("15:20:30.123456")), - tsNtz("2025-06-20T15:20:30.123456")) + timestampToMicros("2025-06-20T15:20:30.123456", UTC)) checkEvaluation(MakeTimestampNTZ(Literal(null, DateType), timeLit("15:20:30.123456")), null) checkEvaluation(MakeTimestampNTZ(dateLit("2025-06-20"), Literal(null, TimeType())), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 2af3111070272..4537277ea6d0f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -1220,6 +1220,60 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } } + test("SPARK-51415: makeTimestamp with days, nanos, and zoneId") { + Seq( + (MIT, -34200000000L), // -09:30 + (PST, -28800000000L), // -08:00 + (UTC, 0L), // +00:00 + (CET, 3600000000L), // +01:00 + (JST, 32400000000L) // +09:00 + ).foreach({ case (zoneId: ZoneId, microsOffset: Long) => + assert(makeTimestamp(0, 0, zoneId) == + 0 - microsOffset) + assert(makeTimestamp(0, localTime(23, 59, 59), zoneId) * NANOS_PER_MICROS == + localTime(23, 59, 59) - microsOffset * NANOS_PER_MICROS) + assert(makeTimestamp(-1, 0, zoneId) == + -1 * MICROS_PER_DAY - microsOffset) + assert(makeTimestamp(-1, localTime(23, 59, 59, 999999), zoneId) == + -1 - microsOffset) + assert(makeTimestamp(days(9999, 12, 31), localTime(23, 59, 59, 999999), zoneId) == + date(9999, 12, 31, 23, 59, 59, 999999) - microsOffset) + assert(makeTimestamp(days(1, 1, 1), localTime(0, 0, 0), zoneId) == + date(1, 1, 1, 0, 0, 0) - microsOffset) + val msg = intercept[DateTimeException] { + makeTimestamp(0, -1, zoneId) + }.getMessage + assert(msg.contains("Invalid value")) + }) + } + + test("SPARK-51415: makeTimestamp with days, nanos, and timezone") { + Seq( + (UTF8String.fromString("-09:30"), -34200000000L), // MIT + (UTF8String.fromString("-08:00"), -28800000000L), // PST + (UTF8String.fromString("+00:00"), 0L), // UTC + (UTF8String.fromString("+01:00"), 3600000000L), // CET + (UTF8String.fromString("+09:00"), 32400000000L) // JST + ).foreach({ case (timezone: UTF8String, microsOffset: Long) => + assert(makeTimestamp(0, 0, timezone) == + 0 - microsOffset) + assert(makeTimestamp(0, localTime(23, 59, 59), timezone) * NANOS_PER_MICROS == + localTime(23, 59, 59) - microsOffset * NANOS_PER_MICROS) + assert(makeTimestamp(-1, 0, timezone) == + -1 * MICROS_PER_DAY - microsOffset) + assert(makeTimestamp(-1, localTime(23, 59, 59, 999999), timezone) == + -1 - microsOffset) + assert(makeTimestamp(days(9999, 12, 31), localTime(23, 59, 59, 999999), timezone) == + date(9999, 12, 31, 23, 59, 59, 999999) - microsOffset) + assert(makeTimestamp(days(1, 1, 1), localTime(0, 0, 0), timezone) == + date(1, 1, 1, 0, 0, 0) - microsOffset) + val msg = intercept[DateTimeException] { + makeTimestamp(0, -1, timezone) + }.getMessage + assert(msg.contains("Invalid value")) + }) + } + test("makeTimestampNTZ") { assert(makeTimestampNTZ(0, 0) == 0) assert(makeTimestampNTZ(0, localTime(23, 59, 59)) * NANOS_PER_MICROS == localTime(23, 59, 59)) diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 3b0b21b9cd776..0499b7fc6519b 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -208,7 +208,7 @@ | org.apache.spark.sql.catalyst.expressions.MakeDate | make_date | SELECT make_date(2013, 7, 15) | struct | | org.apache.spark.sql.catalyst.expressions.MakeInterval | make_interval | SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001) | struct | | org.apache.spark.sql.catalyst.expressions.MakeTime | make_time | SELECT make_time(6, 30, 45.887) | struct | -| org.apache.spark.sql.catalyst.expressions.MakeTimestamp | make_timestamp | SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887) | struct | +| org.apache.spark.sql.catalyst.expressions.MakeTimestampExpressionBuilder | make_timestamp | SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887) | struct | | org.apache.spark.sql.catalyst.expressions.MakeTimestampLTZExpressionBuilder | make_timestamp_ltz | SELECT make_timestamp_ltz(2014, 12, 28, 6, 30, 45.887) | struct | | org.apache.spark.sql.catalyst.expressions.MakeTimestampNTZExpressionBuilder | make_timestamp_ntz | SELECT make_timestamp_ntz(2014, 12, 28, 6, 30, 45.887) | struct | | org.apache.spark.sql.catalyst.expressions.MakeValidUTF8 | make_valid_utf8 | SELECT make_valid_utf8('Spark') | struct | @@ -480,4 +480,4 @@ | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('b1b2b3c1c2','a/b/text()') | structb1b2b3c1c2, a/b/text()):array> | | org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | SELECT xpath_long('12', 'sum(a/b)') | struct12, sum(a/b)):bigint> | | org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | SELECT xpath_short('12', 'sum(a/b)') | struct12, sum(a/b)):smallint> | -| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('bcc','a/c') | structbcc, a/c):string> | +| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | SELECT xpath_string('bcc','a/c') | structbcc, a/c):string> | \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out index 28df8ad68eed9..052bfa3edd8df 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out @@ -1113,6 +1113,188 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, +- OneRowRelation +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), None) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(MIT)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(PST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(UTC)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(CET)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(JST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out index 79f4ec139453a..8792c6e158c84 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out @@ -157,6 +157,188 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, +- OneRowRelation +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), None) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(MIT)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(PST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(UTC)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(CET)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(JST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out index ba6be278efe5a..2a8395a6ee58b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out @@ -157,6 +157,188 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, +- OneRowRelation +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), None) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(MIT)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(PST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(UTC)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(CET)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(JST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out index fce31671cddf2..c0b35c30526ad 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out @@ -158,6 +158,188 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, +- OneRowRelation +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), None) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(MIT)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(PST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(UTC)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(CET)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, true), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(JST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out index ee88233276daa..d0c0f1848a197 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out @@ -158,6 +158,188 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, +- OneRowRelation +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), None) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(MIT)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(PST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(UTC)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(CET)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query analysis +Project [make_timestamp(make_date(2021, 7, 11, false), make_time(6, 30, cast(45.678 as decimal(16,6))), Some(JST)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x] ++- OneRowRelation + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/timestamp.sql index 72ea88a0320e3..73f4c12d3245f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/timestamp.sql @@ -26,6 +26,28 @@ SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999); SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999); SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999); +-- SPARK-51415: timestamp date + time constructor. +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)); +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00'); +SELECT make_timestamp(NULL, TIME'00:00:00'); +SELECT make_timestamp(DATE'1970-01-01', NULL); +SELECT make_timestamp(NULL, NULL); + +-- SPARK-51415: timestamp date + time + timezone constructor. +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT'); +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST'); +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC'); +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET'); +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST'); +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC'); +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC'); +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC'); +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL); +SELECT make_timestamp(NULL, NULL, 'UTC'); +SELECT make_timestamp(NULL, TIME'00:00:00', NULL); +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL); +SELECT make_timestamp(NULL, NULL, NULL); + -- [SPARK-31710] TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS that always create timestamp_ltz select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null); select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.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 ed4ce53d9dfaf..851831052a804 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 @@ -1404,6 +1404,218 @@ org.apache.spark.SparkDateTimeException } +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query schema +struct +-- !query output +2021-07-10 10:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query schema +struct +-- !query output +2021-07-10 14:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out index d995063e23aff..5e457be4e1937 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out @@ -180,6 +180,218 @@ struct NULL +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query schema +struct +-- !query output +2021-07-10 10:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query schema +struct +-- !query output +2021-07-10 14:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out index 27a6b21b9c9a5..79b7f9c1f530e 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out @@ -211,6 +211,218 @@ org.apache.spark.SparkDateTimeException } +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query schema +struct +-- !query output +2021-07-10 10:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query schema +struct +-- !query output +2021-07-10 14:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out index bf4578b803196..ee8a72c315041 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out @@ -211,6 +211,218 @@ org.apache.spark.SparkDateTimeException } +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query schema +struct +-- !query output +2021-07-10 10:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query schema +struct +-- !query output +2021-07-10 14:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out index 0ef7de5f8d546..961549497523f 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out @@ -180,6 +180,218 @@ struct NULL +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678)) +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT') +-- !query schema +struct +-- !query output +2021-07-10 10:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC') +-- !query schema +struct +-- !query output +2021-07-10 23:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST') +-- !query schema +struct +-- !query output +2021-07-10 14:30:45.678 + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC') +-- !query schema +struct +-- !query output +1969-12-31 16:00:00 + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, 'UTC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "make_timestamp(NULL, NULL, 'UTC')" + } ] +} + + +-- !query +SELECT make_timestamp(NULL, TIME'00:00:00', NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "make_timestamp(NULL, TIME'00:00:00', NULL)" + } ] +} + + +-- !query +SELECT make_timestamp(DATE'1970-01-01', NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3, 6, 7]", + "functionName" : "`make_timestamp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_timestamp(NULL, NULL, NULL)" + } ] +} + + -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema