From 9fdf8df25beef3bde460e5aa16fd16dc02c419b6 Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Wed, 3 Apr 2024 11:00:59 -0700
Subject: [PATCH 1/8] [SPARK-47719] LEGACY_TIME_PARSER_POLICY_KEY default to
CORRECTED
---
.../apache/spark/sql/internal/SQLConf.scala | 6 +-
.../sql-tests/inputs/datetime-exception.sql | 3 +
.../inputs/datetime-formatting-exception.sql | 2 +
.../inputs/datetime-parsing-exception.sql | 2 +
.../datetime-parsing-exception.sql.out | 207 ++++++++++++++++++
5 files changed, 217 insertions(+), 3 deletions(-)
create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql
create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql
create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql
create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index af4498274620..8b2ea27fd8a3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -4016,13 +4016,13 @@ object SQLConf {
.doc("When LEGACY, java.text.SimpleDateFormat is used for formatting and parsing " +
"dates/timestamps in a locale-sensitive manner, which is the approach before Spark 3.0. " +
"When set to CORRECTED, classes from java.time.* packages are used for the same purpose. " +
- "The default value is EXCEPTION, RuntimeException is thrown when we will get different " +
- "results.")
+ "When set to EXCEPTION, RuntimeException is thrown when we will get different " +
+ "results. The default is CORRECTED.")
.version("3.0.0")
.stringConf
.transform(_.toUpperCase(Locale.ROOT))
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
- .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+ .createWithDefault(LegacyBehaviorPolicy.CORRECTED.toString)
val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC =
buildConf("spark.sql.legacy.followThreeValuedLogicInArrayExists")
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql
new file mode 100644
index 000000000000..72d0903bf8c6
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql
@@ -0,0 +1,3 @@
+--SET spark.sql.legacy.timeParserPolicy=EXCEPTION
+--IMPORT date.sql
+--IMPORT timestamp.sql
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql
new file mode 100644
index 000000000000..c31c128f0563
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql
@@ -0,0 +1,2 @@
+--SET spark.sql.legacy.timeParserPolicy=EXCEPTION
+--IMPORT datetime-formatting.sql
\ No newline at end of file
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql
new file mode 100644
index 000000000000..83a2eabc6710
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql
@@ -0,0 +1,2 @@
+--SET spark.sql.legacy.timeParserPolicy=EXCEPTION
+--IMPORT datetime-parsing.sql
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out
new file mode 100644
index 000000000000..d45154f97005
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out
@@ -0,0 +1,207 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+select to_timestamp('1', 'y')
+-- !query schema
+struct
+-- !query output
+0001-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('009999', 'y')
+-- !query schema
+struct
+-- !query output
+9999-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('00', 'yy')
+-- !query schema
+struct
+-- !query output
+2000-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('99', 'yy')
+-- !query schema
+struct
+-- !query output
+2099-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('001', 'yyy')
+-- !query schema
+struct
+-- !query output
+0001-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('009999', 'yyy')
+-- !query schema
+struct
+-- !query output
+9999-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('0001', 'yyyy')
+-- !query schema
+struct
+-- !query output
+0001-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('9999', 'yyyy')
+-- !query schema
+struct
+-- !query output
+9999-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('00001', 'yyyyy')
+-- !query schema
+struct
+-- !query output
+0001-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('09999', 'yyyyy')
+-- !query schema
+struct
+-- !query output
+9999-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('000001', 'yyyyyy')
+-- !query schema
+struct
+-- !query output
+0001-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('009999', 'yyyyyy')
+-- !query schema
+struct
+-- !query output
+9999-01-01 00:00:00
+
+
+-- !query
+select to_timestamp('9', 'D')
+-- !query schema
+struct
+-- !query output
+1970-01-09 00:00:00
+
+
+-- !query
+select to_timestamp('300', 'D')
+-- !query schema
+struct
+-- !query output
+1970-10-27 00:00:00
+
+
+-- !query
+select to_timestamp('09', 'DD')
+-- !query schema
+struct
+-- !query output
+1970-01-09 00:00:00
+
+
+-- !query
+select to_timestamp('99', 'DD')
+-- !query schema
+struct
+-- !query output
+1970-04-09 00:00:00
+
+
+-- !query
+select to_timestamp('100', 'DD')
+-- !query schema
+struct
+-- !query output
+1970-04-10 00:00:00
+
+
+-- !query
+select to_timestamp('009', 'DDD')
+-- !query schema
+struct
+-- !query output
+1970-01-09 00:00:00
+
+
+-- !query
+select to_timestamp('365', 'DDD')
+-- !query schema
+struct
+-- !query output
+1970-12-31 00:00:00
+
+
+-- !query
+select to_timestamp('31-365', 'dd-DDD')
+-- !query schema
+struct
+-- !query output
+1970-12-31 00:00:00
+
+
+-- !query
+select to_timestamp('12-365', 'MM-DDD')
+-- !query schema
+struct
+-- !query output
+1970-12-31 00:00:00
+
+
+-- !query
+select to_timestamp('2020-365', 'yyyy-DDD')
+-- !query schema
+struct
+-- !query output
+2020-12-30 00:00:00
+
+
+-- !query
+select to_timestamp('12-31-365', 'MM-dd-DDD')
+-- !query schema
+struct
+-- !query output
+1970-12-31 00:00:00
+
+
+-- !query
+select to_timestamp('2020-30-365', 'yyyy-dd-DDD')
+-- !query schema
+struct
+-- !query output
+2020-12-30 00:00:00
+
+
+-- !query
+select to_timestamp('2020-12-350', 'yyyy-MM-DDD')
+-- !query schema
+struct
+-- !query output
+2020-12-15 00:00:00
+
+
+-- !query
+select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD')
+-- !query schema
+struct
+-- !query output
+2020-12-31 00:00:00
From 8c51461eb8d2a7b9fae7d9061befc0a3f31196ee Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Wed, 3 Apr 2024 11:40:51 -0700
Subject: [PATCH 2/8] Add files
---
.../datetime-exception.sql.out | 2048 ++++++++++++++
.../datetime-parsing-exception.sql.out | 181 ++
.../results/datetime-exception.sql.out | 2439 +++++++++++++++++
3 files changed, 4668 insertions(+)
create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out
create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out
create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out
new file mode 100644
index 000000000000..12a15cc9b896
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out
@@ -0,0 +1,2048 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view date_view as select '2011-11-11' date_str, '1' int_str
+-- !query analysis
+CreateViewCommand `date_view`, select '2011-11-11' date_str, '1' int_str, false, false, LocalTempView, true
+ +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x]
+ +- OneRowRelation
+
+
+-- !query
+select date '2019-01-01\t'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2020-01-01中文'
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2020-01-01中文'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "date '2020-01-01中文'"
+ } ]
+}
+
+
+-- !query
+select make_date(2019, 1, 1), make_date(12, 12, 12)
+-- !query analysis
+Project [make_date(2019, 1, 1, false) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, false) AS make_date(12, 12, 12)#x]
++- OneRowRelation
+
+
+-- !query
+select make_date(2000, 13, 1)
+-- !query analysis
+Project [make_date(2000, 13, 1, false) AS make_date(2000, 13, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select make_date(2000, 1, 33)
+-- !query analysis
+Project [make_date(2000, 1, 33, false) AS make_date(2000, 1, 33)#x]
++- OneRowRelation
+
+
+-- !query
+select date'015'
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'015'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "date'015'"
+ } ]
+}
+
+
+-- !query
+select date'2021-4294967297-11'
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-4294967297-11'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "date'2021-4294967297-11'"
+ } ]
+}
+
+
+-- !query
+select current_date = current_date
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select current_date() = current_date()
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select curdate(1)
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "0",
+ "functionName" : "`curdate`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "curdate(1)"
+ } ]
+}
+
+
+-- !query
+select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
+-- !query analysis
+Project [date_from_unix_date(0) AS date_from_unix_date(0)#x, date_from_unix_date(1000) AS date_from_unix_date(1000)#x, date_from_unix_date(cast(null as int)) AS date_from_unix_date(NULL)#x]
++- OneRowRelation
+
+
+-- !query
+select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd')
+-- !query analysis
+Project [to_date(cast(null as string), None, Some(America/Los_Angeles), false) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), false) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), false) AS to_date(2016-12-31, yyyy-MM-dd)#x]
++- OneRowRelation
+
+
+-- !query
+select to_date("16", "dd")
+-- !query analysis
+Project [to_date(16, Some(dd), Some(America/Los_Angeles), false) AS to_date(16, dd)#x]
++- OneRowRelation
+
+
+-- !query
+select to_date("02-29", "MM-dd")
+-- !query analysis
+Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), false) AS to_date(02-29, MM-dd)#x]
++- OneRowRelation
+
+
+-- !query
+select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null),
+ dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null),
+ weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select next_day("2015-07-23", "Mon")
+-- !query analysis
+Project [next_day(cast(2015-07-23 as date), Mon, false) AS next_day(2015-07-23, Mon)#x]
++- OneRowRelation
+
+
+-- !query
+select next_day("2015-07-23", "xx")
+-- !query analysis
+Project [next_day(cast(2015-07-23 as date), xx, false) AS next_day(2015-07-23, xx)#x]
++- OneRowRelation
+
+
+-- !query
+select next_day("2015-07-23 12:12:12", "Mon")
+-- !query analysis
+Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(2015-07-23 12:12:12, Mon)#x]
++- OneRowRelation
+
+
+-- !query
+select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon")
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon")
+-- !query analysis
+Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x]
++- OneRowRelation
+
+
+-- !query
+select next_day("xx", "Mon")
+-- !query analysis
+Project [next_day(cast(xx as date), Mon, false) AS next_day(xx, Mon)#x]
++- OneRowRelation
+
+
+-- !query
+select next_day(null, "Mon")
+-- !query analysis
+Project [next_day(cast(null as date), Mon, false) AS next_day(NULL, Mon)#x]
++- OneRowRelation
+
+
+-- !query
+select next_day(null, "xx")
+-- !query analysis
+Project [next_day(cast(null as date), xx, false) AS next_day(NULL, xx)#x]
++- OneRowRelation
+
+
+-- !query
+select date_add(date'2011-11-11', 1)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add('2011-11-11', 1)
+-- !query analysis
+Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_add('2011-11-11', 1Y)
+-- !query analysis
+Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_add('2011-11-11', 1S)
+-- !query analysis
+Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_add('2011-11-11', 1L)
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_add('2011-11-11', 1L)"
+ } ]
+}
+
+
+-- !query
+select date_add('2011-11-11', 1.0)
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1.0)"
+ } ]
+}
+
+
+-- !query
+select date_add('2011-11-11', 1E1)
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1E1)"
+ } ]
+}
+
+
+-- !query
+select date_add('2011-11-11', '1')
+-- !query analysis
+Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_add('2011-11-11', '1.2')
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_add"
+ }
+}
+
+
+-- !query
+select date_add(null, 1)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(date'2011-11-11', null)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(timestamp_ltz'2011-11-11 12:12:12', 1)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(timestamp_ntz'2011-11-11 12:12:12', 1)
+-- !query analysis
+Project [date_add(cast(2011-11-11 12:12:12 as date), 1) AS date_add(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_sub(date'2011-11-11', 1)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_sub('2011-11-11', 1)
+-- !query analysis
+Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_sub('2011-11-11', 1Y)
+-- !query analysis
+Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_sub('2011-11-11', 1S)
+-- !query analysis
+Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_sub('2011-11-11', 1L)
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_sub('2011-11-11', 1L)"
+ } ]
+}
+
+
+-- !query
+select date_sub('2011-11-11', 1.0)
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1.0)"
+ } ]
+}
+
+
+-- !query
+select date_sub('2011-11-11', 1E1)
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1E1)"
+ } ]
+}
+
+
+-- !query
+select date_sub(date'2011-11-11', '1')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_sub(date'2011-11-11', '1.2')
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_sub"
+ }
+}
+
+
+-- !query
+select date_sub(null, 1)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_sub(date'2011-11-11', null)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1)
+-- !query analysis
+Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x]
++- OneRowRelation
+
+
+-- !query
+select date_add('2011-11-11', int_str) from date_view
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_add('2011-11-11', int_str)"
+ } ]
+}
+
+
+-- !query
+select date_sub('2011-11-11', int_str) from date_view
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_sub('2011-11-11', int_str)"
+ } ]
+}
+
+
+-- !query
+select date_add(date_str, 1) from date_view
+-- !query analysis
+Project [date_add(cast(date_str#x as date), 1) AS date_add(date_str, 1)#x]
++- SubqueryAlias date_view
+ +- View (`date_view`, [date_str#x, int_str#x])
+ +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x]
+ +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x]
+ +- OneRowRelation
+
+
+-- !query
+select date_sub(date_str, 1) from date_view
+-- !query analysis
+Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#x]
++- SubqueryAlias date_view
+ +- View (`date_view`, [date_str#x, int_str#x])
+ +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x]
+ +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x]
+ +- OneRowRelation
+
+
+-- !query
+select date '2011-11-11' + 1E1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "date '2011-11-11' + 1E1"
+ } ]
+}
+
+
+-- !query
+select date '2001-09-28' + 7Y
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select 7S + date '2001-09-28'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2001-10-01' - 7
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2001-10-01' - date '2001-09-28'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2001-10-01' - '2001-09-28'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2001-09-28\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "date '2001-10-01' - '2001-09-28'"
+ } ]
+}
+
+
+-- !query
+select '2001-10-01' - date '2001-09-28'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2001-09-28' - null
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select null - date '2019-10-06'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_str - date '2001-09-28' from date_view
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2001-09-28' - date_str from date_view
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"date_str\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "date '2001-09-28' - date_str"
+ } ]
+}
+
+
+-- !query
+select date'2011-11-11' + '1'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "date'2011-11-11' + '1'"
+ } ]
+}
+
+
+-- !query
+select '1' + date'2011-11-11'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' + date'2011-11-11'"
+ } ]
+}
+
+
+-- !query
+select date'2011-11-11' + null
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select null + date'2011-11-11'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2012-01-01' - interval '2-2' year to month,
+ date '2011-11-11' - interval '2' day,
+ date '2012-01-01' + interval '-2-2' year to month,
+ date '2011-11-11' + interval '-2' month,
+ - interval '2-2' year to month + date '2012-01-01',
+ interval '-2' day + date '2011-11-11'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select to_date('26/October/2015', 'dd/MMMMM/yyyy')
+-- !query analysis
+Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), false) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
+-- !query analysis
+Project [from_json(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), {"d":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"d":"26/October/2015"})#x]
++- OneRowRelation
+
+
+-- !query
+select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
+-- !query analysis
+Project [from_csv(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x]
++- OneRowRelation
+
+
+-- !query
+select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(MINUTE, -100, date'2022-02-25')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(DAY, 367, date'2022-02-25')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select dateadd(QUARTER, 5, date'2022-02-25')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_add(YEAR, 1, date'2022-02-25')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`dateadd`",
+ "invalidValue" : "'MICROSECOND'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 71,
+ "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')"
+ } ]
+}
+
+
+-- !query
+select date_add('QUARTER', 5, date'2022-02-25')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`date_add`",
+ "invalidValue" : "'QUARTER'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 47,
+ "fragment" : "date_add('QUARTER', 5, date'2022-02-25')"
+ } ]
+}
+
+
+-- !query
+select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select datediff(QUARTER, date'2022-02-25', date'2023-05-25')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_diff(YEAR, date'2022-02-25', date'2023-02-25')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`date_diff`",
+ "invalidValue" : "'MILLISECOND'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 103,
+ "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')"
+ } ]
+}
+
+
+-- !query
+select datediff('YEAR', date'2022-02-25', date'2023-02-25')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`datediff`",
+ "invalidValue" : "'YEAR'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')"
+ } ]
+}
+
+
+-- !query
+select timestamp '2019-01-01\t'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestamp '2019-01-01中文'
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2019-01-01中文'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "timestamp '2019-01-01中文'"
+ } ]
+}
+
+
+-- !query
+select timestamp'4294967297'
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'4294967297'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "timestamp'4294967297'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2021-01-01T12:30:4294967297.123456'
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-01-01T12:30:4294967297.123456'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'"
+ } ]
+}
+
+
+-- !query
+select current_timestamp = current_timestamp
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select current_timestamp() = current_timestamp()
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select localtimestamp() = localtimestamp()
+-- !query analysis
+Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678)
+-- !query analysis
+Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET')
+-- !query analysis
+Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007)
+-- !query analysis
+Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 1)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 60)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 61)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, null)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999)
+-- !query analysis
+Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
+-- !query analysis
+Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23))
+-- !query analysis
+Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null)
+-- !query analysis
+Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null)
+-- !query analysis
+Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000123123)
+-- !query analysis
+Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_SECONDS(-1230219000123123)
+-- !query analysis
+Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_MILLIS(92233720368547758)
+-- !query analysis
+Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_MILLIS(-92233720368547758)
+-- !query analysis
+Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_SECONDS(0.1234567)
+-- !query analysis
+Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x]
++- OneRowRelation
+
+
+-- !query
+select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567))
+-- !query analysis
+Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x]
++- OneRowRelation
+
+
+-- !query
+create temporary view ttf1 as select * from values
+ (1, 2),
+ (2, 3)
+ as ttf1(`current_date`, `current_timestamp`)
+-- !query analysis
+CreateViewCommand `ttf1`, select * from values
+ (1, 2),
+ (2, 3)
+ as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true
+ +- Project [current_date#x, current_timestamp#x]
+ +- SubqueryAlias ttf1
+ +- LocalRelation [current_date#x, current_timestamp#x]
+
+
+-- !query
+select typeof(current_date), typeof(current_timestamp) from ttf1
+-- !query analysis
+Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x]
++- SubqueryAlias ttf1
+ +- View (`ttf1`, [current_date#x, current_timestamp#x])
+ +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x]
+ +- Project [current_date#x, current_timestamp#x]
+ +- SubqueryAlias ttf1
+ +- LocalRelation [current_date#x, current_timestamp#x]
+
+
+-- !query
+create temporary view ttf2 as select * from values
+ (1, 2),
+ (2, 3)
+ as ttf2(a, b)
+-- !query analysis
+CreateViewCommand `ttf2`, select * from values
+ (1, 2),
+ (2, 3)
+ as ttf2(a, b), false, false, LocalTempView, true
+ +- Project [a#x, b#x]
+ +- SubqueryAlias ttf2
+ +- LocalRelation [a#x, b#x]
+
+
+-- !query
+select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select a, b from ttf2 order by a, current_date
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd')
+-- !query analysis
+Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp(1)
+-- !query analysis
+Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query analysis
+Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
+-- !query analysis
+Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
+-- !query analysis
+Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
+-- !query analysis
+Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
+-- !query analysis
+Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
+-- !query analysis
+Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm")
+-- !query analysis
+Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'")
+-- !query analysis
+Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd")
+-- !query analysis
+Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS")
+-- !query analysis
+Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''")
+-- !query analysis
+Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss")
+-- !query analysis
+Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss")
+-- !query analysis
+Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("16", "dd")
+-- !query analysis
+Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(16, dd)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("02-29", "MM-dd")
+-- !query analysis
+Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(02-29, MM-dd)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("2019 40", "yyyy mm")
+-- !query analysis
+Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 40, yyyy mm)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss")
+-- !query analysis
+Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x]
++- OneRowRelation
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2011-11-11 11:11:10\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'"
+ } ]
+}
+
+
+-- !query
+select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2011-11-11 11:11:11\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - null
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select null - timestamp'2011-11-11 11:11:11'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+create temporary view ts_view as select '2011-11-11 11:11:11' str
+-- !query analysis
+CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true
+ +- Project [2011-11-11 11:11:11 AS str#x]
+ +- OneRowRelation
+
+
+-- !query
+select str - timestamp'2011-11-11 11:11:11' from ts_view
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "str - timestamp'2011-11-11 11:11:11'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - str from ts_view
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' - str"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' + '1'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + '1'"
+ } ]
+}
+
+
+-- !query
+select '1' + timestamp'2011-11-11 11:11:11'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"DOUBLE\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "'1' + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' + null
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"VOID\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + null"
+ } ]
+}
+
+
+-- !query
+select null + timestamp'2011-11-11 11:11:11'
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "null + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' + interval '2' day,
+ timestamp'2011-11-11 11:11:11' - interval '2-2' year to month,
+ timestamp'2011-11-11 11:11:11' + interval '-2' second,
+ timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second,
+ - interval 2 years + timestamp'2011-11-11 11:11:11',
+ interval '1 12' day to hour + timestamp'2011-11-11 11:11:11'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date '2012-01-01' - interval 3 hours,
+ date '2012-01-01' + interval '12:12:12' hour to second,
+ interval '2' minute + date '2012-01-01'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG')
+-- !query analysis
+Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE')
+-- !query analysis
+Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
+-- !query analysis
+Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x]
++- OneRowRelation
+
+
+-- !query
+select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
+-- !query analysis
+Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), false) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL]
++- OneRowRelation
+
+
+-- !query
+select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
+-- !query analysis
+Project [from_json(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x]
++- OneRowRelation
+
+
+-- !query
+select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
+-- !query analysis
+Project [from_csv(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x]
++- OneRowRelation
+
+
+-- !query
+select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampadd(YEAR, 1, date'2022-02-15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampadd(SECOND, -1, date'2022-02-15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampadd`",
+ "invalidValue" : "'MONTH'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 64,
+ "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')"
+ } ]
+}
+
+
+-- !query
+select timestampadd('SECOND', -1, date'2022-02-15')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampadd`",
+ "invalidValue" : "'SECOND'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')"
+ } ]
+}
+
+
+-- !query
+select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampdiff`",
+ "invalidValue" : "'MINUTE'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 94,
+ "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')"
+ } ]
+}
+
+
+-- !query
+select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampdiff`",
+ "invalidValue" : "'YEAR'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 64,
+ "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')"
+ } ]
+}
+
+
+-- !query
+select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timediff(DAY, date'2022-02-15', date'2023-02-15')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timediff`",
+ "invalidValue" : "'MINUTE'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 89,
+ "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')"
+ } ]
+}
+
+
+-- !query
+select timediff('YEAR', date'2020-02-15', date'2023-02-15')
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timediff`",
+ "invalidValue" : "'YEAR'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')"
+ } ]
+}
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out
new file mode 100644
index 000000000000..7325f2756949
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out
@@ -0,0 +1,181 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+select to_timestamp('1', 'y')
+-- !query analysis
+Project [to_timestamp(1, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, y)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('009999', 'y')
+-- !query analysis
+Project [to_timestamp(009999, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, y)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('00', 'yy')
+-- !query analysis
+Project [to_timestamp(00, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00, yy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('99', 'yy')
+-- !query analysis
+Project [to_timestamp(99, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, yy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('001', 'yyy')
+-- !query analysis
+Project [to_timestamp(001, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(001, yyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('009999', 'yyy')
+-- !query analysis
+Project [to_timestamp(009999, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('0001', 'yyyy')
+-- !query analysis
+Project [to_timestamp(0001, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(0001, yyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('9999', 'yyyy')
+-- !query analysis
+Project [to_timestamp(9999, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9999, yyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('00001', 'yyyyy')
+-- !query analysis
+Project [to_timestamp(00001, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00001, yyyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('09999', 'yyyyy')
+-- !query analysis
+Project [to_timestamp(09999, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09999, yyyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('000001', 'yyyyyy')
+-- !query analysis
+Project [to_timestamp(000001, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(000001, yyyyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('009999', 'yyyyyy')
+-- !query analysis
+Project [to_timestamp(009999, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyyyyy)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('9', 'D')
+-- !query analysis
+Project [to_timestamp(9, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, D)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('300', 'D')
+-- !query analysis
+Project [to_timestamp(300, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(300, D)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('09', 'DD')
+-- !query analysis
+Project [to_timestamp(09, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09, DD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('99', 'DD')
+-- !query analysis
+Project [to_timestamp(99, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, DD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('100', 'DD')
+-- !query analysis
+Project [to_timestamp(100, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(100, DD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('009', 'DDD')
+-- !query analysis
+Project [to_timestamp(009, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009, DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('365', 'DDD')
+-- !query analysis
+Project [to_timestamp(365, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(365, DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('31-365', 'dd-DDD')
+-- !query analysis
+Project [to_timestamp(31-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(31-365, dd-DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('12-365', 'MM-DDD')
+-- !query analysis
+Project [to_timestamp(12-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-365, MM-DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2020-365', 'yyyy-DDD')
+-- !query analysis
+Project [to_timestamp(2020-365, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-365, yyyy-DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('12-31-365', 'MM-dd-DDD')
+-- !query analysis
+Project [to_timestamp(12-31-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-31-365, MM-dd-DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2020-30-365', 'yyyy-dd-DDD')
+-- !query analysis
+Project [to_timestamp(2020-30-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-30-365, yyyy-dd-DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2020-12-350', 'yyyy-MM-DDD')
+-- !query analysis
+Project [to_timestamp(2020-12-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-350, yyyy-MM-DDD)#x]
++- OneRowRelation
+
+
+-- !query
+select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD')
+-- !query analysis
+Project [to_timestamp(2020-12-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-31-366, yyyy-MM-dd-DDD)#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out
new file mode 100644
index 000000000000..595ef06f10c3
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out
@@ -0,0 +1,2439 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view date_view as select '2011-11-11' date_str, '1' int_str
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select date '2019-01-01\t'
+-- !query schema
+struct
+-- !query output
+2019-01-01
+
+
+-- !query
+select date '2020-01-01中文'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2020-01-01中文'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "date '2020-01-01中文'"
+ } ]
+}
+
+
+-- !query
+select make_date(2019, 1, 1), make_date(12, 12, 12)
+-- !query schema
+struct
+-- !query output
+2019-01-01 0012-12-12
+
+
+-- !query
+select make_date(2000, 13, 1)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select make_date(2000, 1, 33)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date'015'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'015'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "date'015'"
+ } ]
+}
+
+
+-- !query
+select date'2021-4294967297-11'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-4294967297-11'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "date'2021-4294967297-11'"
+ } ]
+}
+
+
+-- !query
+select current_date = current_date
+-- !query schema
+struct<(current_date() = current_date()):boolean>
+-- !query output
+true
+
+
+-- !query
+select current_date() = current_date()
+-- !query schema
+struct<(current_date() = current_date()):boolean>
+-- !query output
+true
+
+
+-- !query
+select curdate(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "0",
+ "functionName" : "`curdate`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "curdate(1)"
+ } ]
+}
+
+
+-- !query
+select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
+-- !query schema
+struct
+-- !query output
+1970-01-01 1972-09-27 NULL
+
+
+-- !query
+select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null)
+-- !query schema
+struct
+-- !query output
+0 18600 NULL
+
+
+-- !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_date("16", "dd")
+-- !query schema
+struct
+-- !query output
+1970-01-16
+
+
+-- !query
+select to_date("02-29", "MM-dd")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null),
+ dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15')
+-- !query schema
+struct
+-- !query output
+7 5 7 NULL 6 6 6
+
+
+-- !query
+select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null),
+ weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15')
+-- !query schema
+struct
+-- !query output
+5 3 5 NULL 4 4 4
+
+
+-- !query
+select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15')
+-- !query schema
+struct
+-- !query output
+1500 1582 1582 1582
+
+
+-- !query
+select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15')
+-- !query schema
+struct
+-- !query output
+1 10 10 10
+
+
+-- !query
+select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15')
+-- !query schema
+struct
+-- !query output
+1 288 288 288
+
+
+-- !query
+select next_day("2015-07-23", "Mon")
+-- !query schema
+struct
+-- !query output
+2015-07-27
+
+
+-- !query
+select next_day("2015-07-23", "xx")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select next_day("2015-07-23 12:12:12", "Mon")
+-- !query schema
+struct
+-- !query output
+2015-07-27
+
+
+-- !query
+select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon")
+-- !query schema
+struct
+-- !query output
+2015-07-27
+
+
+-- !query
+select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon")
+-- !query schema
+struct
+-- !query output
+2015-07-27
+
+
+-- !query
+select next_day("xx", "Mon")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select next_day(null, "Mon")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select next_day(null, "xx")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date_add(date'2011-11-11', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_add('2011-11-11', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_add('2011-11-11', 1Y)
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_add('2011-11-11', 1S)
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_add('2011-11-11', 1L)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_add('2011-11-11', 1L)"
+ } ]
+}
+
+
+-- !query
+select date_add('2011-11-11', 1.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1.0)"
+ } ]
+}
+
+
+-- !query
+select date_add('2011-11-11', 1E1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1E1)"
+ } ]
+}
+
+
+-- !query
+select date_add('2011-11-11', '1')
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_add('2011-11-11', '1.2')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_add"
+ }
+}
+
+
+-- !query
+select date_add(null, 1)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date_add(date'2011-11-11', null)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date_add(timestamp_ltz'2011-11-11 12:12:12', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_add(timestamp_ntz'2011-11-11 12:12:12', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_sub(date'2011-11-11', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_sub('2011-11-11', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_sub('2011-11-11', 1Y)
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_sub('2011-11-11', 1S)
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_sub('2011-11-11', 1L)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_sub('2011-11-11', 1L)"
+ } ]
+}
+
+
+-- !query
+select date_sub('2011-11-11', 1.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1.0)"
+ } ]
+}
+
+
+-- !query
+select date_sub('2011-11-11', 1E1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1E1)"
+ } ]
+}
+
+
+-- !query
+select date_sub(date'2011-11-11', '1')
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_sub(date'2011-11-11', '1.2')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_sub"
+ }
+}
+
+
+-- !query
+select date_sub(null, 1)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date_sub(date'2011-11-11', null)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1)
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date_add('2011-11-11', int_str) from date_view
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_add('2011-11-11', int_str)"
+ } ]
+}
+
+
+-- !query
+select date_sub('2011-11-11', int_str) from date_view
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_sub('2011-11-11', int_str)"
+ } ]
+}
+
+
+-- !query
+select date_add(date_str, 1) from date_view
+-- !query schema
+struct
+-- !query output
+2011-11-12
+
+
+-- !query
+select date_sub(date_str, 1) from date_view
+-- !query schema
+struct
+-- !query output
+2011-11-10
+
+
+-- !query
+select date '2011-11-11' + 1E1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "date '2011-11-11' + 1E1"
+ } ]
+}
+
+
+-- !query
+select date '2001-09-28' + 7Y
+-- !query schema
+struct
+-- !query output
+2001-10-05
+
+
+-- !query
+select 7S + date '2001-09-28'
+-- !query schema
+struct
+-- !query output
+2001-10-05
+
+
+-- !query
+select date '2001-10-01' - 7
+-- !query schema
+struct
+-- !query output
+2001-09-24
+
+
+-- !query
+select date '2001-10-01' - date '2001-09-28'
+-- !query schema
+struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day>
+-- !query output
+3 00:00:00.000000000
+
+
+-- !query
+select date '2001-10-01' - '2001-09-28'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2001-09-28\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "date '2001-10-01' - '2001-09-28'"
+ } ]
+}
+
+
+-- !query
+select '2001-10-01' - date '2001-09-28'
+-- !query schema
+struct<(2001-10-01 - DATE '2001-09-28'):interval day>
+-- !query output
+3 00:00:00.000000000
+
+
+-- !query
+select date '2001-09-28' - null
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select null - date '2019-10-06'
+-- !query schema
+struct<(NULL - DATE '2019-10-06'):interval day>
+-- !query output
+NULL
+
+
+-- !query
+select date_str - date '2001-09-28' from date_view
+-- !query schema
+struct<(date_str - DATE '2001-09-28'):interval day>
+-- !query output
+3696 00:00:00.000000000
+
+
+-- !query
+select date '2001-09-28' - date_str from date_view
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"date_str\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "date '2001-09-28' - date_str"
+ } ]
+}
+
+
+-- !query
+select date'2011-11-11' + '1'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "date'2011-11-11' + '1'"
+ } ]
+}
+
+
+-- !query
+select '1' + date'2011-11-11'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "second",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' + date'2011-11-11'"
+ } ]
+}
+
+
+-- !query
+select date'2011-11-11' + null
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select null + date'2011-11-11'
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select date '2012-01-01' - interval '2-2' year to month,
+ date '2011-11-11' - interval '2' day,
+ date '2012-01-01' + interval '-2-2' year to month,
+ date '2011-11-11' + interval '-2' month,
+ - interval '2-2' year to month + date '2012-01-01',
+ interval '-2' day + date '2011-11-11'
+-- !query schema
+struct
+-- !query output
+2009-11-01 2011-11-09 2009-11-01 2011-09-11 2009-11-01 2011-11-09
+
+
+-- !query
+select to_date('26/October/2015', 'dd/MMMMM/yyyy')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
+
+
+-- !query
+select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
+
+
+-- !query
+select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
+
+
+-- !query
+select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123')
+-- !query schema
+struct
+-- !query output
+2022-02-25 01:02:03.124001
+
+
+-- !query
+select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456')
+-- !query schema
+struct
+-- !query output
+2022-02-25 01:02:03.455
+
+
+-- !query
+select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03')
+-- !query schema
+struct
+-- !query output
+2022-02-25 01:03:01
+
+
+-- !query
+select date_add(MINUTE, -100, date'2022-02-25')
+-- !query schema
+struct
+-- !query output
+2022-02-24 22:20:00
+
+
+-- !query
+select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03')
+-- !query schema
+struct
+-- !query output
+2022-02-25 00:02:03
+
+
+-- !query
+select date_add(DAY, 367, date'2022-02-25')
+-- !query schema
+struct
+-- !query output
+2023-02-27 00:00:00
+
+
+-- !query
+select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03')
+-- !query schema
+struct
+-- !query output
+2022-01-28 01:02:03
+
+
+-- !query
+select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03')
+-- !query schema
+struct
+-- !query output
+2022-01-25 01:02:03
+
+
+-- !query
+select dateadd(QUARTER, 5, date'2022-02-25')
+-- !query schema
+struct
+-- !query output
+2023-05-25 00:00:00
+
+
+-- !query
+select date_add(YEAR, 1, date'2022-02-25')
+-- !query schema
+struct
+-- !query output
+2023-02-25 00:00:00
+
+
+-- !query
+select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`dateadd`",
+ "invalidValue" : "'MICROSECOND'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 71,
+ "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')"
+ } ]
+}
+
+
+-- !query
+select date_add('QUARTER', 5, date'2022-02-25')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`date_add`",
+ "invalidValue" : "'QUARTER'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 47,
+ "fragment" : "date_add('QUARTER', 5, date'2022-02-25')"
+ } ]
+}
+
+
+-- !query
+select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001')
+-- !query schema
+struct
+-- !query output
+1001
+
+
+-- !query
+select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
+-- !query schema
+struct
+-- !query output
+-1
+
+
+-- !query
+select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01')
+-- !query schema
+struct
+-- !query output
+58
+
+
+-- !query
+select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00')
+-- !query schema
+struct
+-- !query output
+-100
+
+
+-- !query
+select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03')
+-- !query schema
+struct
+-- !query output
+-1
+
+
+-- !query
+select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00')
+-- !query schema
+struct
+-- !query output
+367
+
+
+-- !query
+select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03')
+-- !query schema
+struct
+-- !query output
+-4
+
+
+-- !query
+select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03')
+-- !query schema
+struct
+-- !query output
+-1
+
+
+-- !query
+select datediff(QUARTER, date'2022-02-25', date'2023-05-25')
+-- !query schema
+struct
+-- !query output
+5
+
+
+-- !query
+select date_diff(YEAR, date'2022-02-25', date'2023-02-25')
+-- !query schema
+struct
+-- !query output
+1
+
+
+-- !query
+select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`date_diff`",
+ "invalidValue" : "'MILLISECOND'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 103,
+ "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')"
+ } ]
+}
+
+
+-- !query
+select datediff('YEAR', date'2022-02-25', date'2023-02-25')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`datediff`",
+ "invalidValue" : "'YEAR'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')"
+ } ]
+}
+
+
+-- !query
+select timestamp '2019-01-01\t'
+-- !query schema
+struct
+-- !query output
+2019-01-01 00:00:00
+
+
+-- !query
+select timestamp '2019-01-01中文'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2019-01-01中文'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "timestamp '2019-01-01中文'"
+ } ]
+}
+
+
+-- !query
+select timestamp'4294967297'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'4294967297'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "timestamp'4294967297'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2021-01-01T12:30:4294967297.123456'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-01-01T12:30:4294967297.123456'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'"
+ } ]
+}
+
+
+-- !query
+select current_timestamp = current_timestamp
+-- !query schema
+struct<(current_timestamp() = current_timestamp()):boolean>
+-- !query output
+true
+
+
+-- !query
+select current_timestamp() = current_timestamp()
+-- !query schema
+struct<(current_timestamp() = current_timestamp()):boolean>
+-- !query output
+true
+
+
+-- !query
+select localtimestamp() = localtimestamp()
+-- !query schema
+struct<(localtimestamp() = localtimestamp()):boolean>
+-- !query output
+true
+
+
+-- !query
+SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678)
+-- !query schema
+struct
+-- !query output
+2021-07-11 06:30:45.678
+
+
+-- !query
+SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET')
+-- !query schema
+struct
+-- !query output
+2021-07-10 21:30:45.678
+
+
+-- !query
+SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 1)
+-- !query schema
+struct
+-- !query output
+0001-01-01 01:01:01
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 60)
+-- !query schema
+struct
+-- !query output
+0001-01-01 01:02:00
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 61)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, null)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999)
+-- !query schema
+struct
+-- !query output
+0001-01-01 01:01:59.999999
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999)
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
+-- !query schema
+struct
+-- !query output
+2008-12-25 07:30:00 1931-01-07 00:30:00 NULL
+
+
+-- !query
+select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23))
+-- !query schema
+struct
+-- !query output
+1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23
+
+
+-- !query
+select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null)
+-- !query schema
+struct
+-- !query output
+2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL
+
+
+-- !query
+select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null)
+-- !query schema
+struct
+-- !query output
+2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000123123)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_SECONDS(-1230219000123123)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_MILLIS(92233720368547758)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_MILLIS(-92233720368547758)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_SECONDS(0.1234567)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+Rounding necessary
+
+
+-- !query
+select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567))
+-- !query schema
+struct
+-- !query output
+1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456
+
+
+-- !query
+create temporary view ttf1 as select * from values
+ (1, 2),
+ (2, 3)
+ as ttf1(`current_date`, `current_timestamp`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select typeof(current_date), typeof(current_timestamp) from ttf1
+-- !query schema
+struct
+-- !query output
+int int
+int int
+
+
+-- !query
+create temporary view ttf2 as select * from values
+ (1, 2),
+ (2, 3)
+ as ttf2(a, b)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
+-- !query schema
+struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int>
+-- !query output
+true true 1 2
+true true 2 3
+
+
+-- !query
+select a, b from ttf2 order by a, current_date
+-- !query schema
+struct
+-- !query output
+1 2
+2 3
+
+
+-- !query
+select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null)
+-- !query schema
+struct
+-- !query output
+1606833008 1606833008 NULL
+
+
+-- !query
+select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null)
+-- !query schema
+struct
+-- !query output
+1606833008000 1606833008999 NULL
+
+
+-- !query
+select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null)
+-- !query schema
+struct
+-- !query output
+1606833008000000 1606833008999999 NULL
+
+
+-- !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 to_timestamp(1)
+-- !query schema
+struct
+-- !query output
+1969-12-31 16:00:01
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.1
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 03:11:12.123
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 08:11:12.12345
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.123456
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.123456
+
+
+-- !query
+select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.123
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:00
+
+
+-- !query
+select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.12345
+
+
+-- !query
+select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm")
+-- !query schema
+struct
+-- !query output
+0019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'")
+-- !query schema
+struct
+-- !query output
+2019-10-06 00:00:00
+
+
+-- !query
+select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd")
+-- !query schema
+struct
+-- !query output
+2019-10-06 00:00:00
+
+
+-- !query
+select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS")
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12.12
+
+
+-- !query
+select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''")
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss")
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss")
+-- !query schema
+struct
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp("16", "dd")
+-- !query schema
+struct
+-- !query output
+1970-01-16 00:00:00
+
+
+-- !query
+select to_timestamp("02-29", "MM-dd")
+-- !query schema
+struct
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp("2019 40", "yyyy mm")
+-- !query schema
+struct
+-- !query output
+2019-01-01 00:40:00
+
+
+-- !query
+select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss")
+-- !query schema
+struct
+-- !query output
+2019-01-01 10:10:10
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
+-- !query schema
+struct<(TIMESTAMP '2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10'):interval day to second>
+-- !query output
+0 00:00:01.000000000
+
+
+-- !query
+select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'
+-- !query schema
+struct<(DATE '2020-01-01' - TIMESTAMP '2019-10-06 10:11:12.345678'):interval day to second>
+-- !query output
+86 13:48:47.654322000
+
+
+-- !query
+select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'
+-- !query schema
+struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second>
+-- !query output
+-86 13:48:47.654322000
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2011-11-11 11:11:10\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'"
+ } ]
+}
+
+
+-- !query
+select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2011-11-11 11:11:11\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - null
+-- !query schema
+struct<(TIMESTAMP '2011-11-11 11:11:11' - NULL):interval day to second>
+-- !query output
+NULL
+
+
+-- !query
+select null - timestamp'2011-11-11 11:11:11'
+-- !query schema
+struct<(NULL - TIMESTAMP '2011-11-11 11:11:11'):interval day to second>
+-- !query output
+NULL
+
+
+-- !query
+create temporary view ts_view as select '2011-11-11 11:11:11' str
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select str - timestamp'2011-11-11 11:11:11' from ts_view
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "str - timestamp'2011-11-11 11:11:11'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' - str from ts_view
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' - str"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' + '1'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + '1'"
+ } ]
+}
+
+
+-- !query
+select '1' + timestamp'2011-11-11 11:11:11'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"DOUBLE\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "'1' + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' + null
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"VOID\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + null"
+ } ]
+}
+
+
+-- !query
+select null + timestamp'2011-11-11 11:11:11'
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "null + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
+
+
+-- !query
+select timestamp'2011-11-11 11:11:11' + interval '2' day,
+ timestamp'2011-11-11 11:11:11' - interval '2-2' year to month,
+ timestamp'2011-11-11 11:11:11' + interval '-2' second,
+ timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second,
+ - interval 2 years + timestamp'2011-11-11 11:11:11',
+ interval '1 12' day to hour + timestamp'2011-11-11 11:11:11'
+-- !query schema
+struct
+-- !query output
+2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11
+
+
+-- !query
+select date '2012-01-01' - interval 3 hours,
+ date '2012-01-01' + interval '12:12:12' hour to second,
+ interval '2' minute + date '2012-01-01'
+-- !query schema
+struct
+-- !query output
+2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00
+
+
+-- !query
+select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'yyyy-MM-dd GGGGG'"
+ }
+}
+
+
+-- !query
+select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEEE'"
+ }
+}
+
+
+-- !query
+select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEE'"
+ }
+}
+
+
+-- !query
+select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEE'"
+ }
+}
+
+
+-- !query
+select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
+
+
+-- !query
+select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
+
+
+-- !query
+select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03')
+-- !query schema
+struct
+-- !query output
+2022-01-14 01:02:03
+
+
+-- !query
+select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03')
+-- !query schema
+struct
+-- !query output
+2022-02-14 02:00:03
+
+
+-- !query
+select timestampadd(YEAR, 1, date'2022-02-15')
+-- !query schema
+struct
+-- !query output
+2023-02-15 00:00:00
+
+
+-- !query
+select timestampadd(SECOND, -1, date'2022-02-15')
+-- !query schema
+struct
+-- !query output
+2022-02-14 23:59:59
+
+
+-- !query
+select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampadd`",
+ "invalidValue" : "'MONTH'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 64,
+ "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')"
+ } ]
+}
+
+
+-- !query
+select timestampadd('SECOND', -1, date'2022-02-15')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampadd`",
+ "invalidValue" : "'SECOND'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')"
+ } ]
+}
+
+
+-- !query
+select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03')
+-- !query schema
+struct
+-- !query output
+-1
+
+
+-- !query
+select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
+-- !query schema
+struct
+-- !query output
+58
+
+
+-- !query
+select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15')
+-- !query schema
+struct
+-- !query output
+1
+
+
+-- !query
+select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
+-- !query schema
+struct
+-- !query output
+-1
+
+
+-- !query
+select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampdiff`",
+ "invalidValue" : "'MINUTE'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 94,
+ "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')"
+ } ]
+}
+
+
+-- !query
+select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timestampdiff`",
+ "invalidValue" : "'YEAR'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 64,
+ "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')"
+ } ]
+}
+
+
+-- !query
+select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03')
+-- !query schema
+struct
+-- !query output
+-6
+
+
+-- !query
+select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03')
+-- !query schema
+struct
+-- !query output
+10
+
+
+-- !query
+select timediff(DAY, date'2022-02-15', date'2023-02-15')
+-- !query schema
+struct
+-- !query output
+365
+
+
+-- !query
+select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
+-- !query schema
+struct
+-- !query output
+-1
+
+
+-- !query
+select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timediff`",
+ "invalidValue" : "'MINUTE'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 89,
+ "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')"
+ } ]
+}
+
+
+-- !query
+select timediff('YEAR', date'2020-02-15', date'2023-02-15')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "`timediff`",
+ "invalidValue" : "'YEAR'",
+ "parameter" : "`unit`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')"
+ } ]
+}
From ed0d9e2d4e3639864d2785e584a7d29d355f7ca7 Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Wed, 3 Apr 2024 19:57:16 -0700
Subject: [PATCH 3/8] Fix test mismatches,address comments
---
.../apache/spark/sql/ClientE2ETestSuite.scala | 3 +-
docs/sql-migration-guide.md | 2 +
.../datetime-formatting-exception.sql.out | 395 +++++++++++++++
.../ansi/datetime-parsing-invalid.sql.out | 72 ++-
.../datetime-formatting-exception.sql.out | 452 ++++++++++++++++++
.../results/datetime-parsing-invalid.sql.out | 84 +---
.../sql-tests/results/json-functions.sql.out | 24 +-
.../sql-tests/results/xml-functions.sql.out | 24 +-
8 files changed, 905 insertions(+), 151 deletions(-)
create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out
create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
index f2f1571452c0..3f4c855bdb4a 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
@@ -74,7 +74,8 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
for (enrichErrorEnabled <- Seq(false, true)) {
test(s"cause exception - ${enrichErrorEnabled}") {
- withSQLConf("spark.sql.connect.enrichError.enabled" -> enrichErrorEnabled.toString) {
+ withSQLConf("spark.sql.connect.enrichError.enabled" -> enrichErrorEnabled.toString,
+ "spark.sql.legacy.timeParserPolicy" -> "EXCEPTION") {
val ex = intercept[SparkUpgradeException] {
spark
.sql("""
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 13d6702c4cf9..70dedb0a8c97 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -46,6 +46,8 @@ license: |
- Since Spark 4.0, MySQL JDBC datasource will read FLOAT as FloatType, while in Spark 3.5 and previous, it was read as DoubleType. To restore the previous behavior, you can cast the column to the old type.
- Since Spark 4.0, MySQL JDBC datasource will read BIT(n > 1) as BinaryType, while in Spark 3.5 and previous, read as LongType. To restore the previous behavior, set `spark.sql.legacy.mysql.bitArrayMapping.enabled` to `true`.
- Since Spark 4.0, MySQL JDBC datasource will write ShortType as SMALLINT, while in Spark 3.5 and previous, write as INTEGER. To restore the previous behavior, you can replace the column with IntegerType whenever before writing.
+- Since Spark 4.0, The default value for `spark.sql.legacy.ctePrecedencePolicy` has been changed from `EXCEPTION` to `CORRECTED`. Instead of raising an error, inner CTE definitions take precedence over outer definitions.
+- Since Spark 4.0, The default value for `spark.sql.legacy.timeParserPolicy` has been changed from `EXCEPTION` to `CORRECTED`. Instead of raising an error, more values will be accepted for certain [Datetime Patterns for Formatting and Parsing](sql-ref-datetime-pattern.html) by default.
## Upgrading from Spark SQL 3.5.1 to 3.5.2
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out
new file mode 100644
index 000000000000..bc33537b3a8e
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out
@@ -0,0 +1,395 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view v as select col from values
+ (timestamp '1582-06-01 11:33:33.123UTC+080000'),
+ (timestamp '1970-01-01 00:00:00.000Europe/Paris'),
+ (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'),
+ (timestamp '1996-04-01 00:33:33.123Australia/Darwin'),
+ (timestamp '2018-11-17 13:33:33.123Z'),
+ (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'),
+ (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col)
+-- !query analysis
+CreateViewCommand `v`, select col from values
+ (timestamp '1582-06-01 11:33:33.123UTC+080000'),
+ (timestamp '1970-01-01 00:00:00.000Europe/Paris'),
+ (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'),
+ (timestamp '1996-04-01 00:33:33.123Australia/Darwin'),
+ (timestamp '2018-11-17 13:33:33.123Z'),
+ (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'),
+ (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col), false, false, LocalTempView, true
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v
+-- !query analysis
+Project [col#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v
+-- !query analysis
+Project [col#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v
+-- !query analysis
+Project [col#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v
+-- !query analysis
+Project [col#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v
+-- !query analysis
+Project [col#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v
+-- !query analysis
+Project [col#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v
+-- !query analysis
+Project [col#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v
+-- !query analysis
+Project [col#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v
+-- !query analysis
+Project [col#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles'
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v
+-- !query analysis
+Project [col#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v
+-- !query analysis
+Project [col#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v
+-- !query analysis
+Project [col#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v
+-- !query analysis
+Project [col#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v
+-- !query analysis
+Project [col#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v
+-- !query analysis
+Project [col#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v
+-- !query analysis
+Project [col#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v
+-- !query analysis
+Project [col#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v
+-- !query analysis
+Project [col#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v
+-- !query analysis
+Project [col#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v
+-- !query analysis
+Project [col#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v
+-- !query analysis
+Project [col#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v
+-- !query analysis
+Project [col#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v
+-- !query analysis
+Project [col#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v
+-- !query analysis
+Project [col#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v
+-- !query analysis
+Project [col#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v
+-- !query analysis
+Project [col#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v
+-- !query analysis
+Project [col#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v
+-- !query analysis
+Project [col#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v
+-- !query analysis
+Project [col#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x]
++- SubqueryAlias v
+ +- View (`v`, [col#x])
+ +- Project [cast(col#x as timestamp) AS col#x]
+ +- Project [col#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col#x]
+
+
+-- !query
+select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS')
+-- !query analysis
+Project [date_format(cast(2023-08-18 09:13:14.123456 as timestamp), yyyy-MM-dd HH:mm:ss.SSSSSS, Some(America/Los_Angeles)) AS date_format(TIMESTAMP_NTZ '2023-08-18 09:13:14.123456', yyyy-MM-dd HH:mm:ss.SSSSSS)#x, date_format(cast(2023-08-18 09:13:14.123456 as timestamp), yyyy-MM-dd HH:mm:ss.SSSSSS, Some(America/Los_Angeles)) AS date_format(TIMESTAMP_NTZ '2023-08-18 09:13:14.123456', yyyy-MM-dd HH:mm:ss.SSSSSS)#x, date_format(cast(2023-08-18 09:13:14.123456 as timestamp), yyyy-MM-dd HH:mm:ss.SSSSSS, Some(America/Los_Angeles)) AS date_format(TIMESTAMP_NTZ '2023-08-18 09:13:14.123456', yyyy-MM-dd HH:mm:ss.SSSSSS)#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
index 51530c29f79a..514a0c6ae7d3 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
@@ -13,13 +13,13 @@ select to_timestamp('1', 'yy')
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkUpgradeException
+org.apache.spark.SparkDateTimeException
{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
"messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'1'"
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '1' could not be parsed at index 0"
}
}
@@ -45,13 +45,13 @@ select to_timestamp('123', 'yy')
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkUpgradeException
+org.apache.spark.SparkDateTimeException
{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
"messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'123'"
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '123' could not be parsed, unparsed text found at index 2"
}
}
@@ -61,13 +61,13 @@ select to_timestamp('1', 'yyy')
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkUpgradeException
+org.apache.spark.SparkDateTimeException
{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
"messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'1'"
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '1' could not be parsed at index 0"
}
}
@@ -110,13 +110,13 @@ select to_timestamp('9', 'DD')
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkUpgradeException
+org.apache.spark.SparkDateTimeException
{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
"messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'9'"
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '9' could not be parsed at index 0"
}
}
@@ -142,13 +142,13 @@ select to_timestamp('9', 'DDD')
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkUpgradeException
+org.apache.spark.SparkDateTimeException
{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
"messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'9'"
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '9' could not be parsed at index 0"
}
}
@@ -158,13 +158,13 @@ select to_timestamp('99', 'DDD')
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkUpgradeException
+org.apache.spark.SparkDateTimeException
{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
"messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'99'"
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '99' could not be parsed at index 0"
}
}
@@ -284,17 +284,9 @@ org.apache.spark.SparkDateTimeException
-- !query
select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD'))
-- !query schema
-struct<>
+struct>
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'2018-366'"
- }
-}
+{"date":null}
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out
new file mode 100644
index 000000000000..f659dbf0083d
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out
@@ -0,0 +1,452 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view v as select col from values
+ (timestamp '1582-06-01 11:33:33.123UTC+080000'),
+ (timestamp '1970-01-01 00:00:00.000Europe/Paris'),
+ (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'),
+ (timestamp '1996-04-01 00:33:33.123Australia/Darwin'),
+ (timestamp '2018-11-17 13:33:33.123Z'),
+ (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'),
+ (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+1969-12-31 15:00:00 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+1970-12-31 04:59:59.999 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+1996-03-31 07:03:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+2018-11-17 05:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+2019-12-31 09:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+2100-01-01 01:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
+
+
+-- !query
+select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582
+1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969
+1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970
+1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996
+2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018
+2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019
+2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100
+
+
+-- !query
+select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 2 02 2 02 2 02
+1969-12-31 15:00:00 4 04 4 04 4 04
+1970-12-31 04:59:59.999 4 04 4 04 4 04
+1996-03-31 07:03:33.123 1 01 1 01 1 01
+2018-11-17 05:33:33.123 4 04 4 04 4 04
+2019-12-31 09:33:33.123 4 04 4 04 4 04
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter
+1969-12-31 15:00:00 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
+1970-12-31 04:59:59.999 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
+1996-03-31 07:03:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter
+2018-11-17 05:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
+2019-12-31 09:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
+2100-01-01 01:33:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter
+
+
+-- !query
+select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 5 05 May May 5 05 May May 5 05 May May
+1969-12-31 15:00:00 12 12 Dec December 12 12 Dec December 12 12 Dec December
+1970-12-31 04:59:59.999 12 12 Dec December 12 12 Dec December 12 12 Dec December
+1996-03-31 07:03:33.123 3 03 Mar March 3 03 Mar March 3 03 Mar March
+2018-11-17 05:33:33.123 11 11 Nov November 11 11 Nov November 11 11 Nov November
+2019-12-31 09:33:33.123 12 12 Dec December 12 12 Dec December 12 12 Dec December
+2100-01-01 01:33:33.123 1 01 Jan January 1 01 Jan January 1 01 Jan January
+
+
+-- !query
+select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 5 05 5 05 5 05
+1969-12-31 15:00:00 12 12 12 12 12 12
+1970-12-31 04:59:59.999 12 12 12 12 12 12
+1996-03-31 07:03:33.123 3 03 3 03 3 03
+2018-11-17 05:33:33.123 11 11 11 11 11 11
+2019-12-31 09:33:33.123 12 12 12 12 12 12
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 Mon Mon Mon Monday Mon Mon Mon Monday Mon Mon Mon Monday
+1969-12-31 15:00:00 Wed Wed Wed Wednesday Wed Wed Wed Wednesday Wed Wed Wed Wednesday
+1970-12-31 04:59:59.999 Thu Thu Thu Thursday Thu Thu Thu Thursday Thu Thu Thu Thursday
+1996-03-31 07:03:33.123 Sun Sun Sun Sunday Sun Sun Sun Sunday Sun Sun Sun Sunday
+2018-11-17 05:33:33.123 Sat Sat Sat Saturday Sat Sat Sat Saturday Sat Sat Sat Saturday
+2019-12-31 09:33:33.123 Tue Tue Tue Tuesday Tue Tue Tue Tuesday Tue Tue Tue Tuesday
+2100-01-01 01:33:33.123 Fri Fri Fri Friday Fri Fri Fri Friday Fri Fri Fri Friday
+
+
+-- !query
+select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 3 3 3
+1969-12-31 15:00:00 3 3 3
+1970-12-31 04:59:59.999 3 3 3
+1996-03-31 07:03:33.123 3 3 3
+2018-11-17 05:33:33.123 3 3 3
+2019-12-31 09:33:33.123 3 3 3
+2100-01-01 01:33:33.123 1 1 1
+
+
+-- !query
+select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 31 31 31 31 31 31
+1969-12-31 15:00:00 31 31 31 31 31 31
+1970-12-31 04:59:59.999 31 31 31 31 31 31
+1996-03-31 07:03:33.123 31 31 31 31 31 31
+2018-11-17 05:33:33.123 17 17 17 17 17 17
+2019-12-31 09:33:33.123 31 31 31 31 31 31
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles'
+-- !query schema
+struct
+-- !query output
+2100-01-01 01:33:33.123 01 01 01
+
+
+-- !query
+select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 151 151 151 151 151 151
+1969-12-31 15:00:00 365 365 365 365 365 365
+1970-12-31 04:59:59.999 365 365 365 365 365 365
+1996-03-31 07:03:33.123 91 091 91 091 91 091
+2018-11-17 05:33:33.123 321 321 321 321 321 321
+2019-12-31 09:33:33.123 365 365 365 365 365 365
+2100-01-01 01:33:33.123 1 001 1 001 1 001
+
+
+-- !query
+select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 19 19 19 19 19 19
+1969-12-31 15:00:00 15 15 15 15 15 15
+1970-12-31 04:59:59.999 4 04 4 04 4 04
+1996-03-31 07:03:33.123 7 07 7 07 7 07
+2018-11-17 05:33:33.123 5 05 5 05 5 05
+2019-12-31 09:33:33.123 9 09 9 09 9 09
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 7 07 7 07 7 07
+1969-12-31 15:00:00 3 03 3 03 3 03
+1970-12-31 04:59:59.999 4 04 4 04 4 04
+1996-03-31 07:03:33.123 7 07 7 07 7 07
+2018-11-17 05:33:33.123 5 05 5 05 5 05
+2019-12-31 09:33:33.123 9 09 9 09 9 09
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 19 19 19 19 19 19
+1969-12-31 15:00:00 15 15 15 15 15 15
+1970-12-31 04:59:59.999 4 04 4 04 4 04
+1996-03-31 07:03:33.123 7 07 7 07 7 07
+2018-11-17 05:33:33.123 5 05 5 05 5 05
+2019-12-31 09:33:33.123 9 09 9 09 9 09
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 7 07 7 07 7 07
+1969-12-31 15:00:00 3 03 3 03 3 03
+1970-12-31 04:59:59.999 4 04 4 04 4 04
+1996-03-31 07:03:33.123 7 07 7 07 7 07
+2018-11-17 05:33:33.123 5 05 5 05 5 05
+2019-12-31 09:33:33.123 9 09 9 09 9 09
+2100-01-01 01:33:33.123 1 01 1 01 1 01
+
+
+-- !query
+select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 40 40 40 40 40 40
+1969-12-31 15:00:00 0 00 0 00 0 00
+1970-12-31 04:59:59.999 59 59 59 59 59 59
+1996-03-31 07:03:33.123 3 03 3 03 3 03
+2018-11-17 05:33:33.123 33 33 33 33 33 33
+2019-12-31 09:33:33.123 33 33 33 33 33 33
+2100-01-01 01:33:33.123 33 33 33 33 33 33
+
+
+-- !query
+select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 35 35 35 35 35 35
+1969-12-31 15:00:00 0 00 0 00 0 00
+1970-12-31 04:59:59.999 59 59 59 59 59 59
+1996-03-31 07:03:33.123 33 33 33 33 33 33
+2018-11-17 05:33:33.123 33 33 33 33 33 33
+2019-12-31 09:33:33.123 33 33 33 33 33 33
+2100-01-01 01:33:33.123 33 33 33 33 33 33
+
+
+-- !query
+select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
+1969-12-31 15:00:00 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000
+1970-12-31 04:59:59.999 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000
+1996-03-31 07:03:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
+2018-11-17 05:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
+2019-12-31 09:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
+2100-01-01 01:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
+
+
+-- !query
+select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 PM PM PM
+1969-12-31 15:00:00 PM PM PM
+1970-12-31 04:59:59.999 AM AM AM
+1996-03-31 07:03:33.123 AM AM AM
+2018-11-17 05:33:33.123 AM AM AM
+2019-12-31 09:33:33.123 AM AM AM
+2100-01-01 01:33:33.123 AM AM AM
+
+
+-- !query
+select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+1969-12-31 15:00:00 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+1970-12-31 04:59:59.999 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+1996-03-31 07:03:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+2018-11-17 05:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+2019-12-31 09:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+2100-01-01 01:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
+
+
+-- !query
+select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+1969-12-31 15:00:00 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+1970-12-31 04:59:59.999 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+1996-03-31 07:03:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+2018-11-17 05:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+2019-12-31 09:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+2100-01-01 01:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
+
+
+-- !query
+select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 -0752 -0752 -07:52 -0752 -0752 -07:52 -0752 -0752 -07:52
+1969-12-31 15:00:00 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
+1970-12-31 04:59:59.999 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
+1996-03-31 07:03:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
+2018-11-17 05:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
+2019-12-31 09:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
+2100-01-01 01:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
+
+
+-- !query
+select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 -075258 -07:52:58 -075258 -07:52:58 -075258 -07:52:58
+1969-12-31 15:00:00 -0800 -08:00 -0800 -08:00 -0800 -08:00
+1970-12-31 04:59:59.999 -0800 -08:00 -0800 -08:00 -0800 -08:00
+1996-03-31 07:03:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
+2018-11-17 05:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
+2019-12-31 09:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
+2100-01-01 01:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
+
+
+-- !query
+select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58
+1969-12-31 15:00:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
+1970-12-31 04:59:59.999 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
+1996-03-31 07:03:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
+2018-11-17 05:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
+2019-12-31 09:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
+2100-01-01 01:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
+
+
+-- !query
+select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58
+1969-12-31 15:00:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
+1970-12-31 04:59:59.999 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
+1996-03-31 07:03:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
+2018-11-17 05:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
+2019-12-31 09:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
+2100-01-01 01:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
+
+
+-- !query
+select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58
+1969-12-31 15:00:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
+1970-12-31 04:59:59.999 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
+1996-03-31 07:03:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
+2018-11-17 05:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
+2019-12-31 09:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
+2100-01-01 01:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
+
+
+-- !query
+select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 1582-05-31 19:40:35 1582-05-31 19:40:35 1582-05-31 19:40:35
+1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00
+1970-12-31 04:59:59.999 1970-12-31 04:59:59 1970-12-31 04:59:59 1970-12-31 04:59:59
+1996-03-31 07:03:33.123 1996-03-31 07:03:33 1996-03-31 07:03:33 1996-03-31 07:03:33
+2018-11-17 05:33:33.123 2018-11-17 05:33:33 2018-11-17 05:33:33 2018-11-17 05:33:33
+2019-12-31 09:33:33.123 2019-12-31 09:33:33 2019-12-31 09:33:33 2019-12-31 09:33:33
+2100-01-01 01:33:33.123 2100-01-01 01:33:33 2100-01-01 01:33:33 2100-01-01 01:33:33
+
+
+-- !query
+select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+1969-12-31 15:00:00 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+1970-12-31 04:59:59.999 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+1996-03-31 07:03:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+2018-11-17 05:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+2019-12-31 09:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+2100-01-01 01:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
+
+
+-- !query
+select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123 ' ' '
+1969-12-31 15:00:00 ' ' '
+1970-12-31 04:59:59.999 ' ' '
+1996-03-31 07:03:33.123 ' ' '
+2018-11-17 05:33:33.123 ' ' '
+2019-12-31 09:33:33.123 ' ' '
+2100-01-01 01:33:33.123 ' ' '
+
+
+-- !query
+select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v
+-- !query schema
+struct
+-- !query output
+1582-05-31 19:40:35.123
+1969-12-31 15:00:00
+1970-12-31 04:59:59.999
+1996-03-31 07:03:33.123
+2018-11-17 05:33:33.123
+2019-12-31 09:33:33.123
+2100-01-01 01:33:33.123
+
+
+-- !query
+select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd')
+-- !query schema
+struct
+-- !query output
+2023-08-18 2023-08-18 2023-08-18
+
+
+-- !query
+select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ')
+-- !query schema
+struct
+-- !query output
+2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700
+
+
+-- !query
+select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS')
+-- !query schema
+struct
+-- !query output
+2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out
index 6deb3e0fe54e..fffbb2a4e017 100644
--- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out
@@ -11,17 +11,9 @@ long overflow
-- !query
select to_timestamp('1', 'yy')
-- !query schema
-struct<>
+struct
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'1'"
- }
-}
+NULL
-- !query
@@ -35,33 +27,17 @@ NULL
-- !query
select to_timestamp('123', 'yy')
-- !query schema
-struct<>
+struct
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'123'"
- }
-}
+NULL
-- !query
select to_timestamp('1', 'yyy')
-- !query schema
-struct<>
+struct
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'1'"
- }
-}
+NULL
-- !query
@@ -92,17 +68,9 @@ NULL
-- !query
select to_timestamp('9', 'DD')
-- !query schema
-struct<>
+struct
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'9'"
- }
-}
+NULL
-- !query
@@ -116,33 +84,17 @@ NULL
-- !query
select to_timestamp('9', 'DDD')
-- !query schema
-struct<>
+struct
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'9'"
- }
-}
+NULL
-- !query
select to_timestamp('99', 'DDD')
-- !query schema
-struct<>
+struct
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'99'"
- }
-}
+NULL
-- !query
@@ -204,17 +156,9 @@ NULL
-- !query
select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD'))
-- !query schema
-struct<>
+struct>
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'2018-366'"
- }
-}
+{"date":null}
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
index 2232e32077e2..06b8ed88ff88 100644
--- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
@@ -425,17 +425,9 @@ select from_json(
'd date',
map('dateFormat', 'MM-dd'))
-- !query schema
-struct<>
+struct>
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'02-29'"
- }
-}
+{"d":null}
-- !query
@@ -444,17 +436,9 @@ select from_json(
't timestamp',
map('timestampFormat', 'MM-dd'))
-- !query schema
-struct<>
+struct>
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'02-29'"
- }
-}
+{"t":null}
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/xml-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/xml-functions.sql.out
index b03b7d2e70c7..60531b2bfd50 100644
--- a/sql/core/src/test/resources/sql-tests/results/xml-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/xml-functions.sql.out
@@ -357,17 +357,9 @@ select from_xml(
'd date',
map('dateFormat', 'MM-dd'))
-- !query schema
-struct<>
+struct02-29
):struct>
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'02-29'"
- }
-}
+{"d":null}
-- !query
@@ -376,17 +368,9 @@ select from_xml(
't timestamp',
map('timestampFormat', 'MM-dd'))
-- !query schema
-struct<>
+struct02-29):struct>
-- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "datetime" : "'02-29'"
- }
-}
+{"t":null}
-- !query
From 33e81f0c0120616764d42f4bfdeab1e5b4ca1489 Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Thu, 4 Apr 2024 15:27:19 -0700
Subject: [PATCH 4/8] Fix DefaultSqlApiConf
---
.../main/scala/org/apache/spark/sql/internal/SqlApiConf.scala | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala
index 99f7f2a11f2e..813e8228a511 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala
@@ -79,6 +79,6 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf {
override def charVarcharAsString: Boolean = false
override def datetimeJava8ApiEnabled: Boolean = false
override def sessionLocalTimeZone: String = TimeZone.getDefault.getID
- override def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = LegacyBehaviorPolicy.EXCEPTION
+ override def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = LegacyBehaviorPolicy.CORRECTED
override def defaultStringType: StringType = StringType
}
From 703712513b7ad81dcc1873dc7216bc3f9364af2b Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Thu, 4 Apr 2024 17:15:49 -0700
Subject: [PATCH 5/8] Remove unneded new tests, fix other tests, fix docs
---
.../apache/spark/sql/ClientE2ETestSuite.scala | 2 +-
docs/sql-migration-guide.md | 2 +-
.../util/DatetimeFormatterSuite.scala | 61 +-
.../util/TimestampFormatterSuite.scala | 36 +-
.../datetime-exception.sql.out | 2048 --------------
.../datetime-formatting-exception.sql.out | 395 ---
.../datetime-parsing-exception.sql.out | 181 --
.../sql-tests/inputs/datetime-exception.sql | 3 -
.../inputs/datetime-formatting-exception.sql | 2 -
.../inputs/datetime-parsing-exception.sql | 2 -
.../results/datetime-exception.sql.out | 2439 -----------------
.../datetime-formatting-exception.sql.out | 452 ---
.../datetime-parsing-exception.sql.out | 207 --
13 files changed, 53 insertions(+), 5777 deletions(-)
delete mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out
delete mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out
delete mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out
delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql
delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql
delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql
delete mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out
delete mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out
delete mode 100644 sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
index 3f4c855bdb4a..ac5396555547 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
@@ -75,7 +75,7 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
for (enrichErrorEnabled <- Seq(false, true)) {
test(s"cause exception - ${enrichErrorEnabled}") {
withSQLConf("spark.sql.connect.enrichError.enabled" -> enrichErrorEnabled.toString,
- "spark.sql.legacy.timeParserPolicy" -> "EXCEPTION") {
+ "spark.sql.legacy.timeParserPolicy" -> "EXCEPTION") {
val ex = intercept[SparkUpgradeException] {
spark
.sql("""
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 70dedb0a8c97..019728a45f40 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -47,7 +47,7 @@ license: |
- Since Spark 4.0, MySQL JDBC datasource will read BIT(n > 1) as BinaryType, while in Spark 3.5 and previous, read as LongType. To restore the previous behavior, set `spark.sql.legacy.mysql.bitArrayMapping.enabled` to `true`.
- Since Spark 4.0, MySQL JDBC datasource will write ShortType as SMALLINT, while in Spark 3.5 and previous, write as INTEGER. To restore the previous behavior, you can replace the column with IntegerType whenever before writing.
- Since Spark 4.0, The default value for `spark.sql.legacy.ctePrecedencePolicy` has been changed from `EXCEPTION` to `CORRECTED`. Instead of raising an error, inner CTE definitions take precedence over outer definitions.
-- Since Spark 4.0, The default value for `spark.sql.legacy.timeParserPolicy` has been changed from `EXCEPTION` to `CORRECTED`. Instead of raising an error, more values will be accepted for certain [Datetime Patterns for Formatting and Parsing](sql-ref-datetime-pattern.html) by default.
+- Since Spark 4.0, The default value for `spark.sql.legacy.timeParserPolicy` has been changed from `EXCEPTION` to `CORRECTED`. Instead of raising an `INCONSISTENT_BEHAVIOR_CROSS_VERSION` error, `CANNOT_PARSE_TIMESTAMP` will be raised if ANSI mode is enable. `NULL` will be returned if ANSI mode is disabled. See [Datetime Patterns for Formatting and Parsing](sql-ref-datetime-pattern.html).
## Upgrading from Spark SQL 3.5.1 to 3.5.2
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
index 63717a125d2e..bbb825f8398c 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
@@ -20,10 +20,11 @@ package org.apache.spark.sql.catalyst.util
import java.time.DateTimeException
import org.scalatest.matchers.must.Matchers
-
import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUpgradeException}
+
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
+import org.apache.spark.sql.internal.SQLConf
trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {
import DateTimeFormatterHelper._
@@ -99,34 +100,36 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
}
test("SPARK-31939: Fix Parsing day of year when year field pattern is missing") {
- // resolved to queryable LocaleDate or fail directly
- assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
- assertError("yyyy-dd-DD", "2020-02-60",
- "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")
- assertEqual("yyyy-MM-DD", "2020-02-60", date(2020, 2, 29))
- assertError("yyyy-MM-DD", "2020-03-60",
- "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 2020-02-29")
- assertEqual("yyyy-MM-dd-DD", "2020-02-29-60", date(2020, 2, 29))
- assertError("yyyy-MM-dd-DD", "2020-03-01-60",
- "Field DayOfYear 61 differs from DayOfYear 60 derived from 2020-03-01")
- assertEqual("yyyy-DDD", "2020-366", date(2020, 12, 31))
- assertError("yyyy-DDD", "2019-366",
- "Invalid date 'DayOfYear 366' as '2019' is not a leap year")
+ withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "EXCEPTION") {
+ // resolved to queryable LocaleDate or fail directly
+ assertEqual("yyyy-dd-DD", "2020-29-60", date(2020, 2, 29))
+ assertError("yyyy-dd-DD", "2020-02-60",
+ "Field DayOfMonth 29 differs from DayOfMonth 2 derived from 2020-02-29")
+ assertEqual("yyyy-MM-DD", "2020-02-60", date(2020, 2, 29))
+ assertError("yyyy-MM-DD", "2020-03-60",
+ "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 2020-02-29")
+ assertEqual("yyyy-MM-dd-DD", "2020-02-29-60", date(2020, 2, 29))
+ assertError("yyyy-MM-dd-DD", "2020-03-01-60",
+ "Field DayOfYear 61 differs from DayOfYear 60 derived from 2020-03-01")
+ assertEqual("yyyy-DDD", "2020-366", date(2020, 12, 31))
+ assertError("yyyy-DDD", "2019-366",
+ "Invalid date 'DayOfYear 366' as '2019' is not a leap year")
- // unresolved and need to check manually(SPARK-31939 fixed)
- assertEqual("DDD", "365", date(1970, 12, 31))
- assertError("DDD", "366",
- "Invalid date 'DayOfYear 366' as '1970' is not a leap year")
- assertEqual("MM-DD", "03-60", date(1970, 3))
- assertError("MM-DD", "02-60",
- "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 1970-03-01")
- assertEqual("MM-dd-DD", "02-28-59", date(1970, 2, 28))
- assertError("MM-dd-DD", "02-28-60",
- "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 1970-03-01")
- assertError("MM-dd-DD", "02-28-58",
- "Field DayOfMonth 28 differs from DayOfMonth 27 derived from 1970-02-27")
- assertEqual("dd-DD", "28-59", date(1970, 2, 28))
- assertError("dd-DD", "27-59",
- "Field DayOfMonth 27 differs from DayOfMonth 28 derived from 1970-02-28")
+ // unresolved and need to check manually(SPARK-31939 fixed)
+ assertEqual("DDD", "365", date(1970, 12, 31))
+ assertError("DDD", "366",
+ "Invalid date 'DayOfYear 366' as '1970' is not a leap year")
+ assertEqual("MM-DD", "03-60", date(1970, 3))
+ assertError("MM-DD", "02-60",
+ "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 1970-03-01")
+ assertEqual("MM-dd-DD", "02-28-59", date(1970, 2, 28))
+ assertError("MM-dd-DD", "02-28-60",
+ "Field MonthOfYear 2 differs from MonthOfYear 3 derived from 1970-03-01")
+ assertError("MM-dd-DD", "02-28-58",
+ "Field DayOfMonth 28 differs from DayOfMonth 27 derived from 1970-02-27")
+ assertEqual("dd-DD", "28-59", date(1970, 2, 28))
+ assertError("dd-DD", "27-59",
+ "Field DayOfMonth 27 differs from DayOfMonth 28 derived from 1970-02-28")
+ }
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
index 2bc801fbb288..7eac6e0feef6 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
@@ -36,23 +36,25 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite {
override protected def useDateFormatter: Boolean = false
test("parsing timestamps using time zones") {
- val localDate = "2018-12-02T10:11:12.001234"
- val expectedMicros = Map(
- "UTC" -> 1543745472001234L,
- PST.getId -> 1543774272001234L,
- CET.getId -> 1543741872001234L,
- "Africa/Dakar" -> 1543745472001234L,
- "America/Los_Angeles" -> 1543774272001234L,
- "Asia/Urumqi" -> 1543723872001234L,
- "Asia/Hong_Kong" -> 1543716672001234L,
- "Europe/Brussels" -> 1543741872001234L)
- outstandingTimezonesIds.foreach { zoneId =>
- val formatter = TimestampFormatter(
- "yyyy-MM-dd'T'HH:mm:ss.SSSSSS",
- getZoneId(zoneId),
- isParsing = true)
- val microsSinceEpoch = formatter.parse(localDate)
- assert(microsSinceEpoch === expectedMicros(zoneId))
+ withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> "EXCEPTION") {
+ val localDate = "2018-12-02T10:11:12.001234"
+ val expectedMicros = Map(
+ "UTC" -> 1543745472001234L,
+ PST.getId -> 1543774272001234L,
+ CET.getId -> 1543741872001234L,
+ "Africa/Dakar" -> 1543745472001234L,
+ "America/Los_Angeles" -> 1543774272001234L,
+ "Asia/Urumqi" -> 1543723872001234L,
+ "Asia/Hong_Kong" -> 1543716672001234L,
+ "Europe/Brussels" -> 1543741872001234L)
+ outstandingTimezonesIds.foreach { zoneId =>
+ val formatter = TimestampFormatter(
+ "yyyy-MM-dd'T'HH:mm:ss.SSSSSS",
+ getZoneId(zoneId),
+ isParsing = true)
+ val microsSinceEpoch = formatter.parse(localDate)
+ assert(microsSinceEpoch === expectedMicros(zoneId))
+ }
}
}
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out
deleted file mode 100644
index 12a15cc9b896..000000000000
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-exception.sql.out
+++ /dev/null
@@ -1,2048 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-create temporary view date_view as select '2011-11-11' date_str, '1' int_str
--- !query analysis
-CreateViewCommand `date_view`, select '2011-11-11' date_str, '1' int_str, false, false, LocalTempView, true
- +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x]
- +- OneRowRelation
-
-
--- !query
-select date '2019-01-01\t'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2020-01-01中文'
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2020-01-01中文'",
- "valueType" : "\"DATE\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 26,
- "fragment" : "date '2020-01-01中文'"
- } ]
-}
-
-
--- !query
-select make_date(2019, 1, 1), make_date(12, 12, 12)
--- !query analysis
-Project [make_date(2019, 1, 1, false) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, false) AS make_date(12, 12, 12)#x]
-+- OneRowRelation
-
-
--- !query
-select make_date(2000, 13, 1)
--- !query analysis
-Project [make_date(2000, 13, 1, false) AS make_date(2000, 13, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select make_date(2000, 1, 33)
--- !query analysis
-Project [make_date(2000, 1, 33, false) AS make_date(2000, 1, 33)#x]
-+- OneRowRelation
-
-
--- !query
-select date'015'
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'015'",
- "valueType" : "\"DATE\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 16,
- "fragment" : "date'015'"
- } ]
-}
-
-
--- !query
-select date'2021-4294967297-11'
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2021-4294967297-11'",
- "valueType" : "\"DATE\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 31,
- "fragment" : "date'2021-4294967297-11'"
- } ]
-}
-
-
--- !query
-select current_date = current_date
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select current_date() = current_date()
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select curdate(1)
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
- "sqlState" : "42605",
- "messageParameters" : {
- "actualNum" : "1",
- "docroot" : "https://spark.apache.org/docs/latest",
- "expectedNum" : "0",
- "functionName" : "`curdate`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 17,
- "fragment" : "curdate(1)"
- } ]
-}
-
-
--- !query
-select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
--- !query analysis
-Project [date_from_unix_date(0) AS date_from_unix_date(0)#x, date_from_unix_date(1000) AS date_from_unix_date(1000)#x, date_from_unix_date(cast(null as int)) AS date_from_unix_date(NULL)#x]
-+- OneRowRelation
-
-
--- !query
-select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd')
--- !query analysis
-Project [to_date(cast(null as string), None, Some(America/Los_Angeles), false) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), false) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), false) AS to_date(2016-12-31, yyyy-MM-dd)#x]
-+- OneRowRelation
-
-
--- !query
-select to_date("16", "dd")
--- !query analysis
-Project [to_date(16, Some(dd), Some(America/Los_Angeles), false) AS to_date(16, dd)#x]
-+- OneRowRelation
-
-
--- !query
-select to_date("02-29", "MM-dd")
--- !query analysis
-Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), false) AS to_date(02-29, MM-dd)#x]
-+- OneRowRelation
-
-
--- !query
-select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null),
- dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null),
- weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select next_day("2015-07-23", "Mon")
--- !query analysis
-Project [next_day(cast(2015-07-23 as date), Mon, false) AS next_day(2015-07-23, Mon)#x]
-+- OneRowRelation
-
-
--- !query
-select next_day("2015-07-23", "xx")
--- !query analysis
-Project [next_day(cast(2015-07-23 as date), xx, false) AS next_day(2015-07-23, xx)#x]
-+- OneRowRelation
-
-
--- !query
-select next_day("2015-07-23 12:12:12", "Mon")
--- !query analysis
-Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(2015-07-23 12:12:12, Mon)#x]
-+- OneRowRelation
-
-
--- !query
-select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon")
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon")
--- !query analysis
-Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x]
-+- OneRowRelation
-
-
--- !query
-select next_day("xx", "Mon")
--- !query analysis
-Project [next_day(cast(xx as date), Mon, false) AS next_day(xx, Mon)#x]
-+- OneRowRelation
-
-
--- !query
-select next_day(null, "Mon")
--- !query analysis
-Project [next_day(cast(null as date), Mon, false) AS next_day(NULL, Mon)#x]
-+- OneRowRelation
-
-
--- !query
-select next_day(null, "xx")
--- !query analysis
-Project [next_day(cast(null as date), xx, false) AS next_day(NULL, xx)#x]
-+- OneRowRelation
-
-
--- !query
-select date_add(date'2011-11-11', 1)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add('2011-11-11', 1)
--- !query analysis
-Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_add('2011-11-11', 1Y)
--- !query analysis
-Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_add('2011-11-11', 1S)
--- !query analysis
-Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_add('2011-11-11', 1L)
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"BIGINT\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 33,
- "fragment" : "date_add('2011-11-11', 1L)"
- } ]
-}
-
-
--- !query
-select date_add('2011-11-11', 1.0)
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1.0\"",
- "inputType" : "\"DECIMAL(2,1)\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_add('2011-11-11', 1.0)"
- } ]
-}
-
-
--- !query
-select date_add('2011-11-11', 1E1)
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"10.0\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_add('2011-11-11', 1E1)"
- } ]
-}
-
-
--- !query
-select date_add('2011-11-11', '1')
--- !query analysis
-Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_add('2011-11-11', '1.2')
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "date_add"
- }
-}
-
-
--- !query
-select date_add(null, 1)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(date'2011-11-11', null)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(timestamp_ltz'2011-11-11 12:12:12', 1)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(timestamp_ntz'2011-11-11 12:12:12', 1)
--- !query analysis
-Project [date_add(cast(2011-11-11 12:12:12 as date), 1) AS date_add(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_sub(date'2011-11-11', 1)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_sub('2011-11-11', 1)
--- !query analysis
-Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_sub('2011-11-11', 1Y)
--- !query analysis
-Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_sub('2011-11-11', 1S)
--- !query analysis
-Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_sub('2011-11-11', 1L)
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"BIGINT\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 33,
- "fragment" : "date_sub('2011-11-11', 1L)"
- } ]
-}
-
-
--- !query
-select date_sub('2011-11-11', 1.0)
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1.0\"",
- "inputType" : "\"DECIMAL(2,1)\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_sub('2011-11-11', 1.0)"
- } ]
-}
-
-
--- !query
-select date_sub('2011-11-11', 1E1)
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"10.0\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_sub('2011-11-11', 1E1)"
- } ]
-}
-
-
--- !query
-select date_sub(date'2011-11-11', '1')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_sub(date'2011-11-11', '1.2')
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "date_sub"
- }
-}
-
-
--- !query
-select date_sub(null, 1)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_sub(date'2011-11-11', null)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1)
--- !query analysis
-Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x]
-+- OneRowRelation
-
-
--- !query
-select date_add('2011-11-11', int_str) from date_view
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"int_str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, int_str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 38,
- "fragment" : "date_add('2011-11-11', int_str)"
- } ]
-}
-
-
--- !query
-select date_sub('2011-11-11', int_str) from date_view
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"int_str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, int_str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 38,
- "fragment" : "date_sub('2011-11-11', int_str)"
- } ]
-}
-
-
--- !query
-select date_add(date_str, 1) from date_view
--- !query analysis
-Project [date_add(cast(date_str#x as date), 1) AS date_add(date_str, 1)#x]
-+- SubqueryAlias date_view
- +- View (`date_view`, [date_str#x, int_str#x])
- +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x]
- +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x]
- +- OneRowRelation
-
-
--- !query
-select date_sub(date_str, 1) from date_view
--- !query analysis
-Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#x]
-+- SubqueryAlias date_view
- +- View (`date_view`, [date_str#x, int_str#x])
- +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x]
- +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x]
- +- OneRowRelation
-
-
--- !query
-select date '2011-11-11' + 1E1
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"10.0\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 30,
- "fragment" : "date '2011-11-11' + 1E1"
- } ]
-}
-
-
--- !query
-select date '2001-09-28' + 7Y
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select 7S + date '2001-09-28'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2001-10-01' - 7
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2001-10-01' - date '2001-09-28'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2001-10-01' - '2001-09-28'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"2001-09-28\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 39,
- "fragment" : "date '2001-10-01' - '2001-09-28'"
- } ]
-}
-
-
--- !query
-select '2001-10-01' - date '2001-09-28'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2001-09-28' - null
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select null - date '2019-10-06'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_str - date '2001-09-28' from date_view
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2001-09-28' - date_str from date_view
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"date_str\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 35,
- "fragment" : "date '2001-09-28' - date_str"
- } ]
-}
-
-
--- !query
-select date'2011-11-11' + '1'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 29,
- "fragment" : "date'2011-11-11' + '1'"
- } ]
-}
-
-
--- !query
-select '1' + date'2011-11-11'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 29,
- "fragment" : "'1' + date'2011-11-11'"
- } ]
-}
-
-
--- !query
-select date'2011-11-11' + null
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select null + date'2011-11-11'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2012-01-01' - interval '2-2' year to month,
- date '2011-11-11' - interval '2' day,
- date '2012-01-01' + interval '-2-2' year to month,
- date '2011-11-11' + interval '-2' month,
- - interval '2-2' year to month + date '2012-01-01',
- interval '-2' day + date '2011-11-11'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select to_date('26/October/2015', 'dd/MMMMM/yyyy')
--- !query analysis
-Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), false) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
--- !query analysis
-Project [from_json(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), {"d":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"d":"26/October/2015"})#x]
-+- OneRowRelation
-
-
--- !query
-select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
--- !query analysis
-Project [from_csv(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x]
-+- OneRowRelation
-
-
--- !query
-select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(MINUTE, -100, date'2022-02-25')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(DAY, 367, date'2022-02-25')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select dateadd(QUARTER, 5, date'2022-02-25')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_add(YEAR, 1, date'2022-02-25')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`dateadd`",
- "invalidValue" : "'MICROSECOND'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 71,
- "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')"
- } ]
-}
-
-
--- !query
-select date_add('QUARTER', 5, date'2022-02-25')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`date_add`",
- "invalidValue" : "'QUARTER'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 47,
- "fragment" : "date_add('QUARTER', 5, date'2022-02-25')"
- } ]
-}
-
-
--- !query
-select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select datediff(QUARTER, date'2022-02-25', date'2023-05-25')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_diff(YEAR, date'2022-02-25', date'2023-02-25')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`date_diff`",
- "invalidValue" : "'MILLISECOND'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 103,
- "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')"
- } ]
-}
-
-
--- !query
-select datediff('YEAR', date'2022-02-25', date'2023-02-25')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`datediff`",
- "invalidValue" : "'YEAR'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 59,
- "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')"
- } ]
-}
-
-
--- !query
-select timestamp '2019-01-01\t'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestamp '2019-01-01中文'
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2019-01-01中文'",
- "valueType" : "\"TIMESTAMP\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 31,
- "fragment" : "timestamp '2019-01-01中文'"
- } ]
-}
-
-
--- !query
-select timestamp'4294967297'
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'4294967297'",
- "valueType" : "\"TIMESTAMP\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 28,
- "fragment" : "timestamp'4294967297'"
- } ]
-}
-
-
--- !query
-select timestamp'2021-01-01T12:30:4294967297.123456'
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2021-01-01T12:30:4294967297.123456'",
- "valueType" : "\"TIMESTAMP\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 52,
- "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'"
- } ]
-}
-
-
--- !query
-select current_timestamp = current_timestamp
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select current_timestamp() = current_timestamp()
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select localtimestamp() = localtimestamp()
--- !query analysis
-Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678)
--- !query analysis
-Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET')
--- !query analysis
-Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007)
--- !query analysis
-Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 1)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 60)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 61)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, null)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x]
-+- OneRowRelation
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999)
--- !query analysis
-Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
--- !query analysis
-Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23))
--- !query analysis
-Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null)
--- !query analysis
-Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null)
--- !query analysis
-Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_SECONDS(1230219000123123)
--- !query analysis
-Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_SECONDS(-1230219000123123)
--- !query analysis
-Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_MILLIS(92233720368547758)
--- !query analysis
-Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_MILLIS(-92233720368547758)
--- !query analysis
-Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_SECONDS(0.1234567)
--- !query analysis
-Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x]
-+- OneRowRelation
-
-
--- !query
-select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567))
--- !query analysis
-Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x]
-+- OneRowRelation
-
-
--- !query
-create temporary view ttf1 as select * from values
- (1, 2),
- (2, 3)
- as ttf1(`current_date`, `current_timestamp`)
--- !query analysis
-CreateViewCommand `ttf1`, select * from values
- (1, 2),
- (2, 3)
- as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true
- +- Project [current_date#x, current_timestamp#x]
- +- SubqueryAlias ttf1
- +- LocalRelation [current_date#x, current_timestamp#x]
-
-
--- !query
-select typeof(current_date), typeof(current_timestamp) from ttf1
--- !query analysis
-Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x]
-+- SubqueryAlias ttf1
- +- View (`ttf1`, [current_date#x, current_timestamp#x])
- +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x]
- +- Project [current_date#x, current_timestamp#x]
- +- SubqueryAlias ttf1
- +- LocalRelation [current_date#x, current_timestamp#x]
-
-
--- !query
-create temporary view ttf2 as select * from values
- (1, 2),
- (2, 3)
- as ttf2(a, b)
--- !query analysis
-CreateViewCommand `ttf2`, select * from values
- (1, 2),
- (2, 3)
- as ttf2(a, b), false, false, LocalTempView, true
- +- Project [a#x, b#x]
- +- SubqueryAlias ttf2
- +- LocalRelation [a#x, b#x]
-
-
--- !query
-select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select a, b from ttf2 order by a, current_date
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null)
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd')
--- !query analysis
-Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp(1)
--- !query analysis
-Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query analysis
-Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
--- !query analysis
-Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
--- !query analysis
-Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
--- !query analysis
-Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
--- !query analysis
-Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
--- !query analysis
-Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm")
--- !query analysis
-Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'")
--- !query analysis
-Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd")
--- !query analysis
-Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS")
--- !query analysis
-Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''")
--- !query analysis
-Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss")
--- !query analysis
-Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss")
--- !query analysis
-Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("16", "dd")
--- !query analysis
-Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(16, dd)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("02-29", "MM-dd")
--- !query analysis
-Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(02-29, MM-dd)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("2019 40", "yyyy mm")
--- !query analysis
-Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 40, yyyy mm)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss")
--- !query analysis
-Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x]
-+- OneRowRelation
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"2011-11-11 11:11:10\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 61,
- "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'"
- } ]
-}
-
-
--- !query
-select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"2011-11-11 11:11:11\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "first",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 61,
- "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - null
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select null - timestamp'2011-11-11 11:11:11'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-create temporary view ts_view as select '2011-11-11 11:11:11' str
--- !query analysis
-CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true
- +- Project [2011-11-11 11:11:11 AS str#x]
- +- OneRowRelation
-
-
--- !query
-select str - timestamp'2011-11-11 11:11:11' from ts_view
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "first",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "str - timestamp'2011-11-11 11:11:11'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - str from ts_view
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "timestamp'2011-11-11 11:11:11' - str"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' + '1'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"TIMESTAMP\"",
- "right" : "\"DOUBLE\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "timestamp'2011-11-11 11:11:11' + '1'"
- } ]
-}
-
-
--- !query
-select '1' + timestamp'2011-11-11 11:11:11'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"DOUBLE\"",
- "right" : "\"TIMESTAMP\"",
- "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "'1' + timestamp'2011-11-11 11:11:11'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' + null
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"TIMESTAMP\"",
- "right" : "\"VOID\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "timestamp'2011-11-11 11:11:11' + null"
- } ]
-}
-
-
--- !query
-select null + timestamp'2011-11-11 11:11:11'
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"VOID\"",
- "right" : "\"TIMESTAMP\"",
- "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "null + timestamp'2011-11-11 11:11:11'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' + interval '2' day,
- timestamp'2011-11-11 11:11:11' - interval '2-2' year to month,
- timestamp'2011-11-11 11:11:11' + interval '-2' second,
- timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second,
- - interval 2 years + timestamp'2011-11-11 11:11:11',
- interval '1 12' day to hour + timestamp'2011-11-11 11:11:11'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date '2012-01-01' - interval 3 hours,
- date '2012-01-01' + interval '12:12:12' hour to second,
- interval '2' minute + date '2012-01-01'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG')
--- !query analysis
-Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE')
--- !query analysis
-Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
--- !query analysis
-Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x]
-+- OneRowRelation
-
-
--- !query
-select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
--- !query analysis
-Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), false) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL]
-+- OneRowRelation
-
-
--- !query
-select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
--- !query analysis
-Project [from_json(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x]
-+- OneRowRelation
-
-
--- !query
-select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
--- !query analysis
-Project [from_csv(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x]
-+- OneRowRelation
-
-
--- !query
-select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampadd(YEAR, 1, date'2022-02-15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampadd(SECOND, -1, date'2022-02-15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampadd`",
- "invalidValue" : "'MONTH'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 64,
- "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')"
- } ]
-}
-
-
--- !query
-select timestampadd('SECOND', -1, date'2022-02-15')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampadd`",
- "invalidValue" : "'SECOND'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 51,
- "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')"
- } ]
-}
-
-
--- !query
-select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampdiff`",
- "invalidValue" : "'MINUTE'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 94,
- "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')"
- } ]
-}
-
-
--- !query
-select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampdiff`",
- "invalidValue" : "'YEAR'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 64,
- "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')"
- } ]
-}
-
-
--- !query
-select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timediff(DAY, date'2022-02-15', date'2023-02-15')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timediff`",
- "invalidValue" : "'MINUTE'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 89,
- "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')"
- } ]
-}
-
-
--- !query
-select timediff('YEAR', date'2020-02-15', date'2023-02-15')
--- !query analysis
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timediff`",
- "invalidValue" : "'YEAR'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 59,
- "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')"
- } ]
-}
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out
deleted file mode 100644
index bc33537b3a8e..000000000000
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-exception.sql.out
+++ /dev/null
@@ -1,395 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-create temporary view v as select col from values
- (timestamp '1582-06-01 11:33:33.123UTC+080000'),
- (timestamp '1970-01-01 00:00:00.000Europe/Paris'),
- (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'),
- (timestamp '1996-04-01 00:33:33.123Australia/Darwin'),
- (timestamp '2018-11-17 13:33:33.123Z'),
- (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'),
- (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col)
--- !query analysis
-CreateViewCommand `v`, select col from values
- (timestamp '1582-06-01 11:33:33.123UTC+080000'),
- (timestamp '1970-01-01 00:00:00.000Europe/Paris'),
- (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'),
- (timestamp '1996-04-01 00:33:33.123Australia/Darwin'),
- (timestamp '2018-11-17 13:33:33.123Z'),
- (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'),
- (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col), false, false, LocalTempView, true
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v
--- !query analysis
-Project [col#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v
--- !query analysis
-Project [col#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v
--- !query analysis
-Project [col#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v
--- !query analysis
-Project [col#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v
--- !query analysis
-Project [col#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v
--- !query analysis
-Project [col#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v
--- !query analysis
-Project [col#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v
--- !query analysis
-Project [col#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v
--- !query analysis
-Project [col#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles'
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v
--- !query analysis
-Project [col#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v
--- !query analysis
-Project [col#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v
--- !query analysis
-Project [col#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v
--- !query analysis
-Project [col#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v
--- !query analysis
-Project [col#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v
--- !query analysis
-Project [col#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v
--- !query analysis
-Project [col#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v
--- !query analysis
-Project [col#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v
--- !query analysis
-Project [col#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v
--- !query analysis
-Project [col#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v
--- !query analysis
-Project [col#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v
--- !query analysis
-Project [col#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v
--- !query analysis
-Project [col#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v
--- !query analysis
-Project [col#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v
--- !query analysis
-Project [col#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v
--- !query analysis
-Project [col#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v
--- !query analysis
-Project [col#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v
--- !query analysis
-Project [col#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v
--- !query analysis
-Project [col#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v
--- !query analysis
-Project [col#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x]
-+- SubqueryAlias v
- +- View (`v`, [col#x])
- +- Project [cast(col#x as timestamp) AS col#x]
- +- Project [col#x]
- +- SubqueryAlias t
- +- LocalRelation [col#x]
-
-
--- !query
-select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ')
--- !query analysis
-[Analyzer test output redacted due to nondeterminism]
-
-
--- !query
-select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS')
--- !query analysis
-Project [date_format(cast(2023-08-18 09:13:14.123456 as timestamp), yyyy-MM-dd HH:mm:ss.SSSSSS, Some(America/Los_Angeles)) AS date_format(TIMESTAMP_NTZ '2023-08-18 09:13:14.123456', yyyy-MM-dd HH:mm:ss.SSSSSS)#x, date_format(cast(2023-08-18 09:13:14.123456 as timestamp), yyyy-MM-dd HH:mm:ss.SSSSSS, Some(America/Los_Angeles)) AS date_format(TIMESTAMP_NTZ '2023-08-18 09:13:14.123456', yyyy-MM-dd HH:mm:ss.SSSSSS)#x, date_format(cast(2023-08-18 09:13:14.123456 as timestamp), yyyy-MM-dd HH:mm:ss.SSSSSS, Some(America/Los_Angeles)) AS date_format(TIMESTAMP_NTZ '2023-08-18 09:13:14.123456', yyyy-MM-dd HH:mm:ss.SSSSSS)#x]
-+- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out
deleted file mode 100644
index 7325f2756949..000000000000
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-exception.sql.out
+++ /dev/null
@@ -1,181 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-select to_timestamp('1', 'y')
--- !query analysis
-Project [to_timestamp(1, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, y)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('009999', 'y')
--- !query analysis
-Project [to_timestamp(009999, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, y)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('00', 'yy')
--- !query analysis
-Project [to_timestamp(00, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00, yy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('99', 'yy')
--- !query analysis
-Project [to_timestamp(99, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, yy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('001', 'yyy')
--- !query analysis
-Project [to_timestamp(001, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(001, yyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('009999', 'yyy')
--- !query analysis
-Project [to_timestamp(009999, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('0001', 'yyyy')
--- !query analysis
-Project [to_timestamp(0001, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(0001, yyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('9999', 'yyyy')
--- !query analysis
-Project [to_timestamp(9999, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9999, yyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('00001', 'yyyyy')
--- !query analysis
-Project [to_timestamp(00001, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00001, yyyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('09999', 'yyyyy')
--- !query analysis
-Project [to_timestamp(09999, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09999, yyyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('000001', 'yyyyyy')
--- !query analysis
-Project [to_timestamp(000001, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(000001, yyyyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('009999', 'yyyyyy')
--- !query analysis
-Project [to_timestamp(009999, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyyyyy)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('9', 'D')
--- !query analysis
-Project [to_timestamp(9, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, D)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('300', 'D')
--- !query analysis
-Project [to_timestamp(300, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(300, D)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('09', 'DD')
--- !query analysis
-Project [to_timestamp(09, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09, DD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('99', 'DD')
--- !query analysis
-Project [to_timestamp(99, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, DD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('100', 'DD')
--- !query analysis
-Project [to_timestamp(100, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(100, DD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('009', 'DDD')
--- !query analysis
-Project [to_timestamp(009, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009, DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('365', 'DDD')
--- !query analysis
-Project [to_timestamp(365, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(365, DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('31-365', 'dd-DDD')
--- !query analysis
-Project [to_timestamp(31-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(31-365, dd-DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('12-365', 'MM-DDD')
--- !query analysis
-Project [to_timestamp(12-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-365, MM-DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2020-365', 'yyyy-DDD')
--- !query analysis
-Project [to_timestamp(2020-365, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-365, yyyy-DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('12-31-365', 'MM-dd-DDD')
--- !query analysis
-Project [to_timestamp(12-31-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-31-365, MM-dd-DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2020-30-365', 'yyyy-dd-DDD')
--- !query analysis
-Project [to_timestamp(2020-30-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-30-365, yyyy-dd-DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2020-12-350', 'yyyy-MM-DDD')
--- !query analysis
-Project [to_timestamp(2020-12-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-350, yyyy-MM-DDD)#x]
-+- OneRowRelation
-
-
--- !query
-select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD')
--- !query analysis
-Project [to_timestamp(2020-12-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-31-366, yyyy-MM-dd-DDD)#x]
-+- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql
deleted file mode 100644
index 72d0903bf8c6..000000000000
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime-exception.sql
+++ /dev/null
@@ -1,3 +0,0 @@
---SET spark.sql.legacy.timeParserPolicy=EXCEPTION
---IMPORT date.sql
---IMPORT timestamp.sql
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql
deleted file mode 100644
index c31c128f0563..000000000000
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-exception.sql
+++ /dev/null
@@ -1,2 +0,0 @@
---SET spark.sql.legacy.timeParserPolicy=EXCEPTION
---IMPORT datetime-formatting.sql
\ No newline at end of file
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql
deleted file mode 100644
index 83a2eabc6710..000000000000
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-exception.sql
+++ /dev/null
@@ -1,2 +0,0 @@
---SET spark.sql.legacy.timeParserPolicy=EXCEPTION
---IMPORT datetime-parsing.sql
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out
deleted file mode 100644
index 595ef06f10c3..000000000000
--- a/sql/core/src/test/resources/sql-tests/results/datetime-exception.sql.out
+++ /dev/null
@@ -1,2439 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-create temporary view date_view as select '2011-11-11' date_str, '1' int_str
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select date '2019-01-01\t'
--- !query schema
-struct
--- !query output
-2019-01-01
-
-
--- !query
-select date '2020-01-01中文'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2020-01-01中文'",
- "valueType" : "\"DATE\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 26,
- "fragment" : "date '2020-01-01中文'"
- } ]
-}
-
-
--- !query
-select make_date(2019, 1, 1), make_date(12, 12, 12)
--- !query schema
-struct
--- !query output
-2019-01-01 0012-12-12
-
-
--- !query
-select make_date(2000, 13, 1)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select make_date(2000, 1, 33)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date'015'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'015'",
- "valueType" : "\"DATE\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 16,
- "fragment" : "date'015'"
- } ]
-}
-
-
--- !query
-select date'2021-4294967297-11'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2021-4294967297-11'",
- "valueType" : "\"DATE\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 31,
- "fragment" : "date'2021-4294967297-11'"
- } ]
-}
-
-
--- !query
-select current_date = current_date
--- !query schema
-struct<(current_date() = current_date()):boolean>
--- !query output
-true
-
-
--- !query
-select current_date() = current_date()
--- !query schema
-struct<(current_date() = current_date()):boolean>
--- !query output
-true
-
-
--- !query
-select curdate(1)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
- "sqlState" : "42605",
- "messageParameters" : {
- "actualNum" : "1",
- "docroot" : "https://spark.apache.org/docs/latest",
- "expectedNum" : "0",
- "functionName" : "`curdate`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 17,
- "fragment" : "curdate(1)"
- } ]
-}
-
-
--- !query
-select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
--- !query schema
-struct
--- !query output
-1970-01-01 1972-09-27 NULL
-
-
--- !query
-select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null)
--- !query schema
-struct
--- !query output
-0 18600 NULL
-
-
--- !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_date("16", "dd")
--- !query schema
-struct
--- !query output
-1970-01-16
-
-
--- !query
-select to_date("02-29", "MM-dd")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null),
- dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15')
--- !query schema
-struct
--- !query output
-7 5 7 NULL 6 6 6
-
-
--- !query
-select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null),
- weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15')
--- !query schema
-struct
--- !query output
-5 3 5 NULL 4 4 4
-
-
--- !query
-select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15')
--- !query schema
-struct
--- !query output
-1500 1582 1582 1582
-
-
--- !query
-select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15')
--- !query schema
-struct
--- !query output
-1 10 10 10
-
-
--- !query
-select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15')
--- !query schema
-struct
--- !query output
-1 288 288 288
-
-
--- !query
-select next_day("2015-07-23", "Mon")
--- !query schema
-struct
--- !query output
-2015-07-27
-
-
--- !query
-select next_day("2015-07-23", "xx")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select next_day("2015-07-23 12:12:12", "Mon")
--- !query schema
-struct
--- !query output
-2015-07-27
-
-
--- !query
-select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon")
--- !query schema
-struct
--- !query output
-2015-07-27
-
-
--- !query
-select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon")
--- !query schema
-struct
--- !query output
-2015-07-27
-
-
--- !query
-select next_day("xx", "Mon")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select next_day(null, "Mon")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select next_day(null, "xx")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date_add(date'2011-11-11', 1)
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_add('2011-11-11', 1)
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_add('2011-11-11', 1Y)
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_add('2011-11-11', 1S)
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_add('2011-11-11', 1L)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"BIGINT\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 33,
- "fragment" : "date_add('2011-11-11', 1L)"
- } ]
-}
-
-
--- !query
-select date_add('2011-11-11', 1.0)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1.0\"",
- "inputType" : "\"DECIMAL(2,1)\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_add('2011-11-11', 1.0)"
- } ]
-}
-
-
--- !query
-select date_add('2011-11-11', 1E1)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"10.0\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_add('2011-11-11', 1E1)"
- } ]
-}
-
-
--- !query
-select date_add('2011-11-11', '1')
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_add('2011-11-11', '1.2')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "date_add"
- }
-}
-
-
--- !query
-select date_add(null, 1)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date_add(date'2011-11-11', null)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date_add(timestamp_ltz'2011-11-11 12:12:12', 1)
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_add(timestamp_ntz'2011-11-11 12:12:12', 1)
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_sub(date'2011-11-11', 1)
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_sub('2011-11-11', 1)
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_sub('2011-11-11', 1Y)
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_sub('2011-11-11', 1S)
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_sub('2011-11-11', 1L)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"BIGINT\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 33,
- "fragment" : "date_sub('2011-11-11', 1L)"
- } ]
-}
-
-
--- !query
-select date_sub('2011-11-11', 1.0)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1.0\"",
- "inputType" : "\"DECIMAL(2,1)\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_sub('2011-11-11', 1.0)"
- } ]
-}
-
-
--- !query
-select date_sub('2011-11-11', 1E1)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"10.0\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 34,
- "fragment" : "date_sub('2011-11-11', 1E1)"
- } ]
-}
-
-
--- !query
-select date_sub(date'2011-11-11', '1')
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_sub(date'2011-11-11', '1.2')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "date_sub"
- }
-}
-
-
--- !query
-select date_sub(null, 1)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date_sub(date'2011-11-11', null)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1)
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1)
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date_add('2011-11-11', int_str) from date_view
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"int_str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(2011-11-11, int_str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 38,
- "fragment" : "date_add('2011-11-11', int_str)"
- } ]
-}
-
-
--- !query
-select date_sub('2011-11-11', int_str) from date_view
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"int_str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(2011-11-11, int_str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 38,
- "fragment" : "date_sub('2011-11-11', int_str)"
- } ]
-}
-
-
--- !query
-select date_add(date_str, 1) from date_view
--- !query schema
-struct
--- !query output
-2011-11-12
-
-
--- !query
-select date_sub(date_str, 1) from date_view
--- !query schema
-struct
--- !query output
-2011-11-10
-
-
--- !query
-select date '2011-11-11' + 1E1
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"10.0\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 30,
- "fragment" : "date '2011-11-11' + 1E1"
- } ]
-}
-
-
--- !query
-select date '2001-09-28' + 7Y
--- !query schema
-struct
--- !query output
-2001-10-05
-
-
--- !query
-select 7S + date '2001-09-28'
--- !query schema
-struct
--- !query output
-2001-10-05
-
-
--- !query
-select date '2001-10-01' - 7
--- !query schema
-struct
--- !query output
-2001-09-24
-
-
--- !query
-select date '2001-10-01' - date '2001-09-28'
--- !query schema
-struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day>
--- !query output
-3 00:00:00.000000000
-
-
--- !query
-select date '2001-10-01' - '2001-09-28'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"2001-09-28\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 39,
- "fragment" : "date '2001-10-01' - '2001-09-28'"
- } ]
-}
-
-
--- !query
-select '2001-10-01' - date '2001-09-28'
--- !query schema
-struct<(2001-10-01 - DATE '2001-09-28'):interval day>
--- !query output
-3 00:00:00.000000000
-
-
--- !query
-select date '2001-09-28' - null
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select null - date '2019-10-06'
--- !query schema
-struct<(NULL - DATE '2019-10-06'):interval day>
--- !query output
-NULL
-
-
--- !query
-select date_str - date '2001-09-28' from date_view
--- !query schema
-struct<(date_str - DATE '2001-09-28'):interval day>
--- !query output
-3696 00:00:00.000000000
-
-
--- !query
-select date '2001-09-28' - date_str from date_view
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"date_str\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 35,
- "fragment" : "date '2001-09-28' - date_str"
- } ]
-}
-
-
--- !query
-select date'2011-11-11' + '1'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 29,
- "fragment" : "date'2011-11-11' + '1'"
- } ]
-}
-
-
--- !query
-select '1' + date'2011-11-11'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"1\"",
- "inputType" : "\"DOUBLE\"",
- "paramIndex" : "second",
- "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
- "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 29,
- "fragment" : "'1' + date'2011-11-11'"
- } ]
-}
-
-
--- !query
-select date'2011-11-11' + null
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select null + date'2011-11-11'
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select date '2012-01-01' - interval '2-2' year to month,
- date '2011-11-11' - interval '2' day,
- date '2012-01-01' + interval '-2-2' year to month,
- date '2011-11-11' + interval '-2' month,
- - interval '2-2' year to month + date '2012-01-01',
- interval '-2' day + date '2011-11-11'
--- !query schema
-struct
--- !query output
-2009-11-01 2011-11-09 2009-11-01 2011-09-11 2009-11-01 2011-11-09
-
-
--- !query
-select to_date('26/October/2015', 'dd/MMMMM/yyyy')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd/MMMMM/yyyy'"
- }
-}
-
-
--- !query
-select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd/MMMMM/yyyy'"
- }
-}
-
-
--- !query
-select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy'))
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd/MMMMM/yyyy'"
- }
-}
-
-
--- !query
-select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123')
--- !query schema
-struct
--- !query output
-2022-02-25 01:02:03.124001
-
-
--- !query
-select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456')
--- !query schema
-struct
--- !query output
-2022-02-25 01:02:03.455
-
-
--- !query
-select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03')
--- !query schema
-struct
--- !query output
-2022-02-25 01:03:01
-
-
--- !query
-select date_add(MINUTE, -100, date'2022-02-25')
--- !query schema
-struct
--- !query output
-2022-02-24 22:20:00
-
-
--- !query
-select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03')
--- !query schema
-struct
--- !query output
-2022-02-25 00:02:03
-
-
--- !query
-select date_add(DAY, 367, date'2022-02-25')
--- !query schema
-struct
--- !query output
-2023-02-27 00:00:00
-
-
--- !query
-select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03')
--- !query schema
-struct
--- !query output
-2022-01-28 01:02:03
-
-
--- !query
-select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03')
--- !query schema
-struct
--- !query output
-2022-01-25 01:02:03
-
-
--- !query
-select dateadd(QUARTER, 5, date'2022-02-25')
--- !query schema
-struct
--- !query output
-2023-05-25 00:00:00
-
-
--- !query
-select date_add(YEAR, 1, date'2022-02-25')
--- !query schema
-struct
--- !query output
-2023-02-25 00:00:00
-
-
--- !query
-select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`dateadd`",
- "invalidValue" : "'MICROSECOND'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 71,
- "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')"
- } ]
-}
-
-
--- !query
-select date_add('QUARTER', 5, date'2022-02-25')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`date_add`",
- "invalidValue" : "'QUARTER'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 47,
- "fragment" : "date_add('QUARTER', 5, date'2022-02-25')"
- } ]
-}
-
-
--- !query
-select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001')
--- !query schema
-struct
--- !query output
-1001
-
-
--- !query
-select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
--- !query schema
-struct
--- !query output
--1
-
-
--- !query
-select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01')
--- !query schema
-struct
--- !query output
-58
-
-
--- !query
-select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00')
--- !query schema
-struct
--- !query output
--100
-
-
--- !query
-select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03')
--- !query schema
-struct
--- !query output
--1
-
-
--- !query
-select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00')
--- !query schema
-struct
--- !query output
-367
-
-
--- !query
-select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03')
--- !query schema
-struct
--- !query output
--4
-
-
--- !query
-select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03')
--- !query schema
-struct
--- !query output
--1
-
-
--- !query
-select datediff(QUARTER, date'2022-02-25', date'2023-05-25')
--- !query schema
-struct
--- !query output
-5
-
-
--- !query
-select date_diff(YEAR, date'2022-02-25', date'2023-02-25')
--- !query schema
-struct
--- !query output
-1
-
-
--- !query
-select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`date_diff`",
- "invalidValue" : "'MILLISECOND'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 103,
- "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')"
- } ]
-}
-
-
--- !query
-select datediff('YEAR', date'2022-02-25', date'2023-02-25')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`datediff`",
- "invalidValue" : "'YEAR'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 59,
- "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')"
- } ]
-}
-
-
--- !query
-select timestamp '2019-01-01\t'
--- !query schema
-struct
--- !query output
-2019-01-01 00:00:00
-
-
--- !query
-select timestamp '2019-01-01中文'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2019-01-01中文'",
- "valueType" : "\"TIMESTAMP\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 31,
- "fragment" : "timestamp '2019-01-01中文'"
- } ]
-}
-
-
--- !query
-select timestamp'4294967297'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'4294967297'",
- "valueType" : "\"TIMESTAMP\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 28,
- "fragment" : "timestamp'4294967297'"
- } ]
-}
-
-
--- !query
-select timestamp'2021-01-01T12:30:4294967297.123456'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_TYPED_LITERAL",
- "sqlState" : "42604",
- "messageParameters" : {
- "value" : "'2021-01-01T12:30:4294967297.123456'",
- "valueType" : "\"TIMESTAMP\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 52,
- "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'"
- } ]
-}
-
-
--- !query
-select current_timestamp = current_timestamp
--- !query schema
-struct<(current_timestamp() = current_timestamp()):boolean>
--- !query output
-true
-
-
--- !query
-select current_timestamp() = current_timestamp()
--- !query schema
-struct<(current_timestamp() = current_timestamp()):boolean>
--- !query output
-true
-
-
--- !query
-select localtimestamp() = localtimestamp()
--- !query schema
-struct<(localtimestamp() = localtimestamp()):boolean>
--- !query output
-true
-
-
--- !query
-SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678)
--- !query schema
-struct
--- !query output
-2021-07-11 06:30:45.678
-
-
--- !query
-SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET')
--- !query schema
-struct
--- !query output
-2021-07-10 21:30:45.678
-
-
--- !query
-SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 1)
--- !query schema
-struct
--- !query output
-0001-01-01 01:01:01
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 60)
--- !query schema
-struct
--- !query output
-0001-01-01 01:02:00
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 61)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, null)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999)
--- !query schema
-struct
--- !query output
-0001-01-01 01:01:59.999999
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999)
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
--- !query schema
-struct
--- !query output
-2008-12-25 07:30:00 1931-01-07 00:30:00 NULL
-
-
--- !query
-select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23))
--- !query schema
-struct
--- !query output
-1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23
-
-
--- !query
-select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null)
--- !query schema
-struct
--- !query output
-2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL
-
-
--- !query
-select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null)
--- !query schema
-struct
--- !query output
-2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL
-
-
--- !query
-select TIMESTAMP_SECONDS(1230219000123123)
--- !query schema
-struct<>
--- !query output
-java.lang.ArithmeticException
-long overflow
-
-
--- !query
-select TIMESTAMP_SECONDS(-1230219000123123)
--- !query schema
-struct<>
--- !query output
-java.lang.ArithmeticException
-long overflow
-
-
--- !query
-select TIMESTAMP_MILLIS(92233720368547758)
--- !query schema
-struct<>
--- !query output
-java.lang.ArithmeticException
-long overflow
-
-
--- !query
-select TIMESTAMP_MILLIS(-92233720368547758)
--- !query schema
-struct<>
--- !query output
-java.lang.ArithmeticException
-long overflow
-
-
--- !query
-select TIMESTAMP_SECONDS(0.1234567)
--- !query schema
-struct<>
--- !query output
-java.lang.ArithmeticException
-Rounding necessary
-
-
--- !query
-select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567))
--- !query schema
-struct
--- !query output
-1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456
-
-
--- !query
-create temporary view ttf1 as select * from values
- (1, 2),
- (2, 3)
- as ttf1(`current_date`, `current_timestamp`)
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select typeof(current_date), typeof(current_timestamp) from ttf1
--- !query schema
-struct
--- !query output
-int int
-int int
-
-
--- !query
-create temporary view ttf2 as select * from values
- (1, 2),
- (2, 3)
- as ttf2(a, b)
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
--- !query schema
-struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int>
--- !query output
-true true 1 2
-true true 2 3
-
-
--- !query
-select a, b from ttf2 order by a, current_date
--- !query schema
-struct
--- !query output
-1 2
-2 3
-
-
--- !query
-select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null)
--- !query schema
-struct
--- !query output
-1606833008 1606833008 NULL
-
-
--- !query
-select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null)
--- !query schema
-struct
--- !query output
-1606833008000 1606833008999 NULL
-
-
--- !query
-select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null)
--- !query schema
-struct
--- !query output
-1606833008000000 1606833008999999 NULL
-
-
--- !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 to_timestamp(1)
--- !query schema
-struct
--- !query output
-1969-12-31 16:00:01
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.1
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.12
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 03:11:12.123
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.1234
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 08:11:12.12345
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.123456
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.123456
-
-
--- !query
-select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.1234
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.123
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12
-
-
--- !query
-select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.12
-
-
--- !query
-select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:00
-
-
--- !query
-select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.12345
-
-
--- !query
-select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.1234
-
-
--- !query
-select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm")
--- !query schema
-struct
--- !query output
-0019-10-06 10:11:12.1234
-
-
--- !query
-select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'")
--- !query schema
-struct
--- !query output
-2019-10-06 00:00:00
-
-
--- !query
-select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd")
--- !query schema
-struct
--- !query output
-2019-10-06 00:00:00
-
-
--- !query
-select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS")
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12.12
-
-
--- !query
-select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''")
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12
-
-
--- !query
-select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss")
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12
-
-
--- !query
-select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss")
--- !query schema
-struct
--- !query output
-2019-10-06 10:11:12
-
-
--- !query
-select to_timestamp("16", "dd")
--- !query schema
-struct
--- !query output
-1970-01-16 00:00:00
-
-
--- !query
-select to_timestamp("02-29", "MM-dd")
--- !query schema
-struct
--- !query output
-NULL
-
-
--- !query
-select to_timestamp("2019 40", "yyyy mm")
--- !query schema
-struct
--- !query output
-2019-01-01 00:40:00
-
-
--- !query
-select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss")
--- !query schema
-struct
--- !query output
-2019-01-01 10:10:10
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
--- !query schema
-struct<(TIMESTAMP '2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10'):interval day to second>
--- !query output
-0 00:00:01.000000000
-
-
--- !query
-select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'
--- !query schema
-struct<(DATE '2020-01-01' - TIMESTAMP '2019-10-06 10:11:12.345678'):interval day to second>
--- !query output
-86 13:48:47.654322000
-
-
--- !query
-select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'
--- !query schema
-struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second>
--- !query output
--86 13:48:47.654322000
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"2011-11-11 11:11:10\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 61,
- "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'"
- } ]
-}
-
-
--- !query
-select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"2011-11-11 11:11:11\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "first",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 61,
- "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - null
--- !query schema
-struct<(TIMESTAMP '2011-11-11 11:11:11' - NULL):interval day to second>
--- !query output
-NULL
-
-
--- !query
-select null - timestamp'2011-11-11 11:11:11'
--- !query schema
-struct<(NULL - TIMESTAMP '2011-11-11 11:11:11'):interval day to second>
--- !query output
-NULL
-
-
--- !query
-create temporary view ts_view as select '2011-11-11 11:11:11' str
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select str - timestamp'2011-11-11 11:11:11' from ts_view
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "first",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "str - timestamp'2011-11-11 11:11:11'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' - str from ts_view
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"str\"",
- "inputType" : "\"STRING\"",
- "paramIndex" : "second",
- "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "timestamp'2011-11-11 11:11:11' - str"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' + '1'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"TIMESTAMP\"",
- "right" : "\"DOUBLE\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "timestamp'2011-11-11 11:11:11' + '1'"
- } ]
-}
-
-
--- !query
-select '1' + timestamp'2011-11-11 11:11:11'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"DOUBLE\"",
- "right" : "\"TIMESTAMP\"",
- "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "'1' + timestamp'2011-11-11 11:11:11'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' + null
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"TIMESTAMP\"",
- "right" : "\"VOID\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "timestamp'2011-11-11 11:11:11' + null"
- } ]
-}
-
-
--- !query
-select null + timestamp'2011-11-11 11:11:11'
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"VOID\"",
- "right" : "\"TIMESTAMP\"",
- "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "null + timestamp'2011-11-11 11:11:11'"
- } ]
-}
-
-
--- !query
-select timestamp'2011-11-11 11:11:11' + interval '2' day,
- timestamp'2011-11-11 11:11:11' - interval '2-2' year to month,
- timestamp'2011-11-11 11:11:11' + interval '-2' second,
- timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second,
- - interval 2 years + timestamp'2011-11-11 11:11:11',
- interval '1 12' day to hour + timestamp'2011-11-11 11:11:11'
--- !query schema
-struct
--- !query output
-2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11
-
-
--- !query
-select date '2012-01-01' - interval 3 hours,
- date '2012-01-01' + interval '12:12:12' hour to second,
- interval '2' minute + date '2012-01-01'
--- !query schema
-struct
--- !query output
-2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00
-
-
--- !query
-select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'yyyy-MM-dd GGGGG'"
- }
-}
-
-
--- !query
-select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd MM yyyy EEEEEE'"
- }
-}
-
-
--- !query
-select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd MM yyyy EEEEE'"
- }
-}
-
-
--- !query
-select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd MM yyyy EEEEE'"
- }
-}
-
-
--- !query
-select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd/MMMMM/yyyy'"
- }
-}
-
-
--- !query
-select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy'))
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkUpgradeException
-{
- "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
- "sqlState" : "42K0B",
- "messageParameters" : {
- "config" : "\"spark.sql.legacy.timeParserPolicy\"",
- "docroot" : "https://spark.apache.org/docs/latest",
- "pattern" : "'dd/MMMMM/yyyy'"
- }
-}
-
-
--- !query
-select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03')
--- !query schema
-struct
--- !query output
-2022-01-14 01:02:03
-
-
--- !query
-select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03')
--- !query schema
-struct
--- !query output
-2022-02-14 02:00:03
-
-
--- !query
-select timestampadd(YEAR, 1, date'2022-02-15')
--- !query schema
-struct
--- !query output
-2023-02-15 00:00:00
-
-
--- !query
-select timestampadd(SECOND, -1, date'2022-02-15')
--- !query schema
-struct
--- !query output
-2022-02-14 23:59:59
-
-
--- !query
-select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampadd`",
- "invalidValue" : "'MONTH'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 64,
- "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')"
- } ]
-}
-
-
--- !query
-select timestampadd('SECOND', -1, date'2022-02-15')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampadd`",
- "invalidValue" : "'SECOND'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 51,
- "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')"
- } ]
-}
-
-
--- !query
-select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03')
--- !query schema
-struct
--- !query output
--1
-
-
--- !query
-select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
--- !query schema
-struct
--- !query output
-58
-
-
--- !query
-select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15')
--- !query schema
-struct
--- !query output
-1
-
-
--- !query
-select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
--- !query schema
-struct
--- !query output
--1
-
-
--- !query
-select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampdiff`",
- "invalidValue" : "'MINUTE'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 94,
- "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')"
- } ]
-}
-
-
--- !query
-select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timestampdiff`",
- "invalidValue" : "'YEAR'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 64,
- "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')"
- } ]
-}
-
-
--- !query
-select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03')
--- !query schema
-struct
--- !query output
--6
-
-
--- !query
-select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03')
--- !query schema
-struct
--- !query output
-10
-
-
--- !query
-select timediff(DAY, date'2022-02-15', date'2023-02-15')
--- !query schema
-struct
--- !query output
-365
-
-
--- !query
-select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59')
--- !query schema
-struct
--- !query output
--1
-
-
--- !query
-select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timediff`",
- "invalidValue" : "'MINUTE'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 89,
- "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')"
- } ]
-}
-
-
--- !query
-select timediff('YEAR', date'2020-02-15', date'2023-02-15')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-{
- "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT",
- "sqlState" : "22023",
- "messageParameters" : {
- "functionName" : "`timediff`",
- "invalidValue" : "'YEAR'",
- "parameter" : "`unit`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 59,
- "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')"
- } ]
-}
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out
deleted file mode 100644
index f659dbf0083d..000000000000
--- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-exception.sql.out
+++ /dev/null
@@ -1,452 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-create temporary view v as select col from values
- (timestamp '1582-06-01 11:33:33.123UTC+080000'),
- (timestamp '1970-01-01 00:00:00.000Europe/Paris'),
- (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'),
- (timestamp '1996-04-01 00:33:33.123Australia/Darwin'),
- (timestamp '2018-11-17 13:33:33.123Z'),
- (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'),
- (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col)
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-1969-12-31 15:00:00 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-1970-12-31 04:59:59.999 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-1996-03-31 07:03:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-2018-11-17 05:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-2019-12-31 09:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-2100-01-01 01:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini
-
-
--- !query
-select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582
-1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969
-1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970
-1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996
-2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018
-2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019
-2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100
-
-
--- !query
-select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 2 02 2 02 2 02
-1969-12-31 15:00:00 4 04 4 04 4 04
-1970-12-31 04:59:59.999 4 04 4 04 4 04
-1996-03-31 07:03:33.123 1 01 1 01 1 01
-2018-11-17 05:33:33.123 4 04 4 04 4 04
-2019-12-31 09:33:33.123 4 04 4 04 4 04
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter
-1969-12-31 15:00:00 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
-1970-12-31 04:59:59.999 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
-1996-03-31 07:03:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter
-2018-11-17 05:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
-2019-12-31 09:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter
-2100-01-01 01:33:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter
-
-
--- !query
-select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 5 05 May May 5 05 May May 5 05 May May
-1969-12-31 15:00:00 12 12 Dec December 12 12 Dec December 12 12 Dec December
-1970-12-31 04:59:59.999 12 12 Dec December 12 12 Dec December 12 12 Dec December
-1996-03-31 07:03:33.123 3 03 Mar March 3 03 Mar March 3 03 Mar March
-2018-11-17 05:33:33.123 11 11 Nov November 11 11 Nov November 11 11 Nov November
-2019-12-31 09:33:33.123 12 12 Dec December 12 12 Dec December 12 12 Dec December
-2100-01-01 01:33:33.123 1 01 Jan January 1 01 Jan January 1 01 Jan January
-
-
--- !query
-select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 5 05 5 05 5 05
-1969-12-31 15:00:00 12 12 12 12 12 12
-1970-12-31 04:59:59.999 12 12 12 12 12 12
-1996-03-31 07:03:33.123 3 03 3 03 3 03
-2018-11-17 05:33:33.123 11 11 11 11 11 11
-2019-12-31 09:33:33.123 12 12 12 12 12 12
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 Mon Mon Mon Monday Mon Mon Mon Monday Mon Mon Mon Monday
-1969-12-31 15:00:00 Wed Wed Wed Wednesday Wed Wed Wed Wednesday Wed Wed Wed Wednesday
-1970-12-31 04:59:59.999 Thu Thu Thu Thursday Thu Thu Thu Thursday Thu Thu Thu Thursday
-1996-03-31 07:03:33.123 Sun Sun Sun Sunday Sun Sun Sun Sunday Sun Sun Sun Sunday
-2018-11-17 05:33:33.123 Sat Sat Sat Saturday Sat Sat Sat Saturday Sat Sat Sat Saturday
-2019-12-31 09:33:33.123 Tue Tue Tue Tuesday Tue Tue Tue Tuesday Tue Tue Tue Tuesday
-2100-01-01 01:33:33.123 Fri Fri Fri Friday Fri Fri Fri Friday Fri Fri Fri Friday
-
-
--- !query
-select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 3 3 3
-1969-12-31 15:00:00 3 3 3
-1970-12-31 04:59:59.999 3 3 3
-1996-03-31 07:03:33.123 3 3 3
-2018-11-17 05:33:33.123 3 3 3
-2019-12-31 09:33:33.123 3 3 3
-2100-01-01 01:33:33.123 1 1 1
-
-
--- !query
-select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 31 31 31 31 31 31
-1969-12-31 15:00:00 31 31 31 31 31 31
-1970-12-31 04:59:59.999 31 31 31 31 31 31
-1996-03-31 07:03:33.123 31 31 31 31 31 31
-2018-11-17 05:33:33.123 17 17 17 17 17 17
-2019-12-31 09:33:33.123 31 31 31 31 31 31
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles'
--- !query schema
-struct
--- !query output
-2100-01-01 01:33:33.123 01 01 01
-
-
--- !query
-select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 151 151 151 151 151 151
-1969-12-31 15:00:00 365 365 365 365 365 365
-1970-12-31 04:59:59.999 365 365 365 365 365 365
-1996-03-31 07:03:33.123 91 091 91 091 91 091
-2018-11-17 05:33:33.123 321 321 321 321 321 321
-2019-12-31 09:33:33.123 365 365 365 365 365 365
-2100-01-01 01:33:33.123 1 001 1 001 1 001
-
-
--- !query
-select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 19 19 19 19 19 19
-1969-12-31 15:00:00 15 15 15 15 15 15
-1970-12-31 04:59:59.999 4 04 4 04 4 04
-1996-03-31 07:03:33.123 7 07 7 07 7 07
-2018-11-17 05:33:33.123 5 05 5 05 5 05
-2019-12-31 09:33:33.123 9 09 9 09 9 09
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 7 07 7 07 7 07
-1969-12-31 15:00:00 3 03 3 03 3 03
-1970-12-31 04:59:59.999 4 04 4 04 4 04
-1996-03-31 07:03:33.123 7 07 7 07 7 07
-2018-11-17 05:33:33.123 5 05 5 05 5 05
-2019-12-31 09:33:33.123 9 09 9 09 9 09
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 19 19 19 19 19 19
-1969-12-31 15:00:00 15 15 15 15 15 15
-1970-12-31 04:59:59.999 4 04 4 04 4 04
-1996-03-31 07:03:33.123 7 07 7 07 7 07
-2018-11-17 05:33:33.123 5 05 5 05 5 05
-2019-12-31 09:33:33.123 9 09 9 09 9 09
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 7 07 7 07 7 07
-1969-12-31 15:00:00 3 03 3 03 3 03
-1970-12-31 04:59:59.999 4 04 4 04 4 04
-1996-03-31 07:03:33.123 7 07 7 07 7 07
-2018-11-17 05:33:33.123 5 05 5 05 5 05
-2019-12-31 09:33:33.123 9 09 9 09 9 09
-2100-01-01 01:33:33.123 1 01 1 01 1 01
-
-
--- !query
-select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 40 40 40 40 40 40
-1969-12-31 15:00:00 0 00 0 00 0 00
-1970-12-31 04:59:59.999 59 59 59 59 59 59
-1996-03-31 07:03:33.123 3 03 3 03 3 03
-2018-11-17 05:33:33.123 33 33 33 33 33 33
-2019-12-31 09:33:33.123 33 33 33 33 33 33
-2100-01-01 01:33:33.123 33 33 33 33 33 33
-
-
--- !query
-select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 35 35 35 35 35 35
-1969-12-31 15:00:00 0 00 0 00 0 00
-1970-12-31 04:59:59.999 59 59 59 59 59 59
-1996-03-31 07:03:33.123 33 33 33 33 33 33
-2018-11-17 05:33:33.123 33 33 33 33 33 33
-2019-12-31 09:33:33.123 33 33 33 33 33 33
-2100-01-01 01:33:33.123 33 33 33 33 33 33
-
-
--- !query
-select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
-1969-12-31 15:00:00 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000
-1970-12-31 04:59:59.999 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000
-1996-03-31 07:03:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
-2018-11-17 05:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
-2019-12-31 09:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
-2100-01-01 01:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000
-
-
--- !query
-select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 PM PM PM
-1969-12-31 15:00:00 PM PM PM
-1970-12-31 04:59:59.999 AM AM AM
-1996-03-31 07:03:33.123 AM AM AM
-2018-11-17 05:33:33.123 AM AM AM
-2019-12-31 09:33:33.123 AM AM AM
-2100-01-01 01:33:33.123 AM AM AM
-
-
--- !query
-select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-1969-12-31 15:00:00 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-1970-12-31 04:59:59.999 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-1996-03-31 07:03:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-2018-11-17 05:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-2019-12-31 09:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-2100-01-01 01:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles
-
-
--- !query
-select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-1969-12-31 15:00:00 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-1970-12-31 04:59:59.999 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-1996-03-31 07:03:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-2018-11-17 05:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-2019-12-31 09:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-2100-01-01 01:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time
-
-
--- !query
-select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 -0752 -0752 -07:52 -0752 -0752 -07:52 -0752 -0752 -07:52
-1969-12-31 15:00:00 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
-1970-12-31 04:59:59.999 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
-1996-03-31 07:03:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
-2018-11-17 05:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
-2019-12-31 09:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
-2100-01-01 01:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00
-
-
--- !query
-select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 -075258 -07:52:58 -075258 -07:52:58 -075258 -07:52:58
-1969-12-31 15:00:00 -0800 -08:00 -0800 -08:00 -0800 -08:00
-1970-12-31 04:59:59.999 -0800 -08:00 -0800 -08:00 -0800 -08:00
-1996-03-31 07:03:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
-2018-11-17 05:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
-2019-12-31 09:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
-2100-01-01 01:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00
-
-
--- !query
-select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58
-1969-12-31 15:00:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
-1970-12-31 04:59:59.999 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
-1996-03-31 07:03:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
-2018-11-17 05:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
-2019-12-31 09:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
-2100-01-01 01:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00
-
-
--- !query
-select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58
-1969-12-31 15:00:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
-1970-12-31 04:59:59.999 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
-1996-03-31 07:03:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
-2018-11-17 05:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
-2019-12-31 09:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
-2100-01-01 01:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00
-
-
--- !query
-select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58
-1969-12-31 15:00:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
-1970-12-31 04:59:59.999 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
-1996-03-31 07:03:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
-2018-11-17 05:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
-2019-12-31 09:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
-2100-01-01 01:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00
-
-
--- !query
-select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 1582-05-31 19:40:35 1582-05-31 19:40:35 1582-05-31 19:40:35
-1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00
-1970-12-31 04:59:59.999 1970-12-31 04:59:59 1970-12-31 04:59:59 1970-12-31 04:59:59
-1996-03-31 07:03:33.123 1996-03-31 07:03:33 1996-03-31 07:03:33 1996-03-31 07:03:33
-2018-11-17 05:33:33.123 2018-11-17 05:33:33 2018-11-17 05:33:33 2018-11-17 05:33:33
-2019-12-31 09:33:33.123 2019-12-31 09:33:33 2019-12-31 09:33:33 2019-12-31 09:33:33
-2100-01-01 01:33:33.123 2100-01-01 01:33:33 2100-01-01 01:33:33 2100-01-01 01:33:33
-
-
--- !query
-select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-1969-12-31 15:00:00 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-1970-12-31 04:59:59.999 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-1996-03-31 07:03:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-2018-11-17 05:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-2019-12-31 09:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-2100-01-01 01:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV
-
-
--- !query
-select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123 ' ' '
-1969-12-31 15:00:00 ' ' '
-1970-12-31 04:59:59.999 ' ' '
-1996-03-31 07:03:33.123 ' ' '
-2018-11-17 05:33:33.123 ' ' '
-2019-12-31 09:33:33.123 ' ' '
-2100-01-01 01:33:33.123 ' ' '
-
-
--- !query
-select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v
--- !query schema
-struct
--- !query output
-1582-05-31 19:40:35.123
-1969-12-31 15:00:00
-1970-12-31 04:59:59.999
-1996-03-31 07:03:33.123
-2018-11-17 05:33:33.123
-2019-12-31 09:33:33.123
-2100-01-01 01:33:33.123
-
-
--- !query
-select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd')
--- !query schema
-struct
--- !query output
-2023-08-18 2023-08-18 2023-08-18
-
-
--- !query
-select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ')
--- !query schema
-struct
--- !query output
-2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700
-
-
--- !query
-select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS')
--- !query schema
-struct
--- !query output
-2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out
deleted file mode 100644
index d45154f97005..000000000000
--- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-exception.sql.out
+++ /dev/null
@@ -1,207 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-select to_timestamp('1', 'y')
--- !query schema
-struct
--- !query output
-0001-01-01 00:00:00
-
-
--- !query
-select to_timestamp('009999', 'y')
--- !query schema
-struct
--- !query output
-9999-01-01 00:00:00
-
-
--- !query
-select to_timestamp('00', 'yy')
--- !query schema
-struct
--- !query output
-2000-01-01 00:00:00
-
-
--- !query
-select to_timestamp('99', 'yy')
--- !query schema
-struct
--- !query output
-2099-01-01 00:00:00
-
-
--- !query
-select to_timestamp('001', 'yyy')
--- !query schema
-struct
--- !query output
-0001-01-01 00:00:00
-
-
--- !query
-select to_timestamp('009999', 'yyy')
--- !query schema
-struct
--- !query output
-9999-01-01 00:00:00
-
-
--- !query
-select to_timestamp('0001', 'yyyy')
--- !query schema
-struct
--- !query output
-0001-01-01 00:00:00
-
-
--- !query
-select to_timestamp('9999', 'yyyy')
--- !query schema
-struct
--- !query output
-9999-01-01 00:00:00
-
-
--- !query
-select to_timestamp('00001', 'yyyyy')
--- !query schema
-struct
--- !query output
-0001-01-01 00:00:00
-
-
--- !query
-select to_timestamp('09999', 'yyyyy')
--- !query schema
-struct
--- !query output
-9999-01-01 00:00:00
-
-
--- !query
-select to_timestamp('000001', 'yyyyyy')
--- !query schema
-struct
--- !query output
-0001-01-01 00:00:00
-
-
--- !query
-select to_timestamp('009999', 'yyyyyy')
--- !query schema
-struct
--- !query output
-9999-01-01 00:00:00
-
-
--- !query
-select to_timestamp('9', 'D')
--- !query schema
-struct
--- !query output
-1970-01-09 00:00:00
-
-
--- !query
-select to_timestamp('300', 'D')
--- !query schema
-struct
--- !query output
-1970-10-27 00:00:00
-
-
--- !query
-select to_timestamp('09', 'DD')
--- !query schema
-struct
--- !query output
-1970-01-09 00:00:00
-
-
--- !query
-select to_timestamp('99', 'DD')
--- !query schema
-struct
--- !query output
-1970-04-09 00:00:00
-
-
--- !query
-select to_timestamp('100', 'DD')
--- !query schema
-struct
--- !query output
-1970-04-10 00:00:00
-
-
--- !query
-select to_timestamp('009', 'DDD')
--- !query schema
-struct
--- !query output
-1970-01-09 00:00:00
-
-
--- !query
-select to_timestamp('365', 'DDD')
--- !query schema
-struct
--- !query output
-1970-12-31 00:00:00
-
-
--- !query
-select to_timestamp('31-365', 'dd-DDD')
--- !query schema
-struct
--- !query output
-1970-12-31 00:00:00
-
-
--- !query
-select to_timestamp('12-365', 'MM-DDD')
--- !query schema
-struct
--- !query output
-1970-12-31 00:00:00
-
-
--- !query
-select to_timestamp('2020-365', 'yyyy-DDD')
--- !query schema
-struct
--- !query output
-2020-12-30 00:00:00
-
-
--- !query
-select to_timestamp('12-31-365', 'MM-dd-DDD')
--- !query schema
-struct
--- !query output
-1970-12-31 00:00:00
-
-
--- !query
-select to_timestamp('2020-30-365', 'yyyy-dd-DDD')
--- !query schema
-struct
--- !query output
-2020-12-30 00:00:00
-
-
--- !query
-select to_timestamp('2020-12-350', 'yyyy-MM-DDD')
--- !query schema
-struct
--- !query output
-2020-12-15 00:00:00
-
-
--- !query
-select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD')
--- !query schema
-struct
--- !query output
-2020-12-31 00:00:00
From 8b786fe0a9367c940ae446bef29b3ffc828a2438 Mon Sep 17 00:00:00 2001
From: Wenchen Fan
Date: Fri, 5 Apr 2024 10:06:21 +0800
Subject: [PATCH 6/8] Update DatetimeFormatterSuite.scala
---
.../apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
index bbb825f8398c..a60602065d74 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala
@@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.util
import java.time.DateTimeException
import org.scalatest.matchers.must.Matchers
-import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUpgradeException}
+import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUpgradeException}
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC}
import org.apache.spark.sql.internal.SQLConf
From e78e5ebb3eab95f6705bf1116846e2318345cf63 Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Thu, 4 Apr 2024 19:38:14 -0700
Subject: [PATCH 7/8] Fix scala style and another testcase
---
.../test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala | 3 ++-
.../apache/spark/sql/catalyst/util/DateFormatterSuite.scala | 2 +-
2 files changed, 3 insertions(+), 2 deletions(-)
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
index ac5396555547..95ee69d2a47d 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
@@ -74,7 +74,8 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
for (enrichErrorEnabled <- Seq(false, true)) {
test(s"cause exception - ${enrichErrorEnabled}") {
- withSQLConf("spark.sql.connect.enrichError.enabled" -> enrichErrorEnabled.toString,
+ withSQLConf(
+ "spark.sql.connect.enrichError.enabled" -> enrichErrorEnabled.toString,
"spark.sql.legacy.timeParserPolicy" -> "EXCEPTION") {
val ex = intercept[SparkUpgradeException] {
spark
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala
index 20b1ade81f59..110a63981ba2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala
@@ -185,7 +185,7 @@ class DateFormatterSuite extends DatetimeFormatterSuite {
val formatter = DateFormatter("MM-dd")
// The date parser in 2.4 accepts 1970-02-29 and turn it into 1970-03-01, so we should get a
// SparkUpgradeException here.
- intercept[SparkUpgradeException](formatter.parse("02-29"))
+ intercept[DateTimeException](formatter.parse("02-29"))
}
test("SPARK-36418: default parsing w/o pattern") {
From 0792f3efcca1f26802a346324d43cae42881c901 Mon Sep 17 00:00:00 2001
From: Serge Rielau
Date: Fri, 5 Apr 2024 07:55:35 -0700
Subject: [PATCH 8/8] Fix spark connect testcase
---
python/pyspark/sql/tests/connect/test_connect_session.py | 1 +
1 file changed, 1 insertion(+)
diff --git a/python/pyspark/sql/tests/connect/test_connect_session.py b/python/pyspark/sql/tests/connect/test_connect_session.py
index bebe2cfc2923..52312540742a 100644
--- a/python/pyspark/sql/tests/connect/test_connect_session.py
+++ b/python/pyspark/sql/tests/connect/test_connect_session.py
@@ -109,6 +109,7 @@ def test_error_enrichment_jvm_stacktrace(self):
{
"spark.sql.connect.enrichError.enabled": True,
"spark.sql.pyspark.jvmStacktrace.enabled": False,
+ "spark.sql.legacy.timeParserPolicy": "EXCEPTION",
}
):
with self.sql_conf({"spark.sql.connect.serverStacktrace.enabled": False}):