This is an automated email from the ASF dual-hosted git repository.

gengliang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 9814cf8  [SPARK-35889][SQL] Support adding TimestampWithoutTZ with 
Interval types
9814cf8 is described below

commit 9814cf88533c049036cee5f6d62346f237dcec19
Author: Gengliang Wang <gengli...@apache.org>
AuthorDate: Fri Jun 25 19:58:42 2021 +0800

    [SPARK-35889][SQL] Support adding TimestampWithoutTZ with Interval types
    
    ### What changes were proposed in this pull request?
    
    Supprot the following operations:
    
    - TimestampWithoutTZ + Calendar interval
    - TimestampWithoutTZ + Year-Month interval
    - TimestampWithoutTZ + Daytime interval
    
    ### Why are the changes needed?
    
    Support basic '+' operator for timestamp without time zone type.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No, the timestamp without time zone type is not release yet.
    
    ### How was this patch tested?
    
    Unit tests
    
    Closes #33076 from gengliangwang/addForNewTS.
    
    Authored-by: Gengliang Wang <gengli...@apache.org>
    Signed-off-by: Gengliang Wang <gengli...@apache.org>
---
 .../spark/sql/catalyst/analysis/Analyzer.scala     |   6 +-
 .../catalyst/expressions/datetimeExpressions.scala |  28 ++-
 .../apache/spark/sql/types/AbstractDataType.scala  |   8 +
 .../expressions/DateExpressionsSuite.scala         | 245 +++++++++++----------
 .../test/resources/sql-tests/inputs/datetime.sql   |  11 +
 .../sql-tests/results/ansi/datetime.sql.out        |  76 ++++++-
 .../sql-tests/results/datetime-legacy.sql.out      |  76 ++++++-
 .../resources/sql-tests/results/datetime.sql.out   |  76 ++++++-
 .../typeCoercion/native/dateTimeOperations.sql.out |  54 ++---
 9 files changed, 424 insertions(+), 156 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 0a3bd09..6737ed5 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -357,8 +357,10 @@ class Analyzer(override val catalogManager: CatalogManager)
           case (_: DayTimeIntervalType, DateType) => TimeAdd(Cast(r, 
TimestampType), l)
           case (DateType, _: YearMonthIntervalType) => DateAddYMInterval(l, r)
           case (_: YearMonthIntervalType, DateType) => DateAddYMInterval(r, l)
-          case (TimestampType, _: YearMonthIntervalType) => 
TimestampAddYMInterval(l, r)
-          case (_: YearMonthIntervalType, TimestampType) => 
TimestampAddYMInterval(r, l)
+          case (TimestampType | TimestampWithoutTZType, _: 
YearMonthIntervalType) =>
+            TimestampAddYMInterval(l, r)
+          case (_: YearMonthIntervalType, TimestampType | 
TimestampWithoutTZType) =>
+            TimestampAddYMInterval(r, l)
           case (CalendarIntervalType, CalendarIntervalType) |
                (_: DayTimeIntervalType, _: DayTimeIntervalType) => a
           case (DateType, CalendarIntervalType) => DateAddInterval(l, r, 
ansiEnabled = f)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 63f6c03..d84b6eb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -59,6 +59,11 @@ trait TimeZoneAwareExpression extends Expression {
   def withTimeZone(timeZoneId: String): TimeZoneAwareExpression
 
   @transient lazy val zoneId: ZoneId = DateTimeUtils.getZoneId(timeZoneId.get)
+
+  def zoneIdForType(dataType: DataType): ZoneId = dataType match {
+    case _: TimestampWithoutTZType => java.time.ZoneOffset.UTC
+    case _ => zoneId
+  }
 }
 
 trait TimestampFormatterHelper extends TimeZoneAwareExpression {
@@ -1446,23 +1451,25 @@ case class TimeAdd(start: Expression, interval: 
Expression, timeZoneId: Option[S
   override def toString: String = s"$left + $right"
   override def sql: String = s"${left.sql} + ${right.sql}"
   override def inputTypes: Seq[AbstractDataType] =
-    Seq(TimestampType, TypeCollection(CalendarIntervalType, 
DayTimeIntervalType))
+    Seq(TypeCollection.AllTimestampTypes, TypeCollection(CalendarIntervalType, 
DayTimeIntervalType))
 
-  override def dataType: DataType = TimestampType
+  override def dataType: DataType = start.dataType
 
   override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
     copy(timeZoneId = Option(timeZoneId))
 
+  @transient private lazy val zoneIdInEval: ZoneId = 
zoneIdForType(left.dataType)
+
   override def nullSafeEval(start: Any, interval: Any): Any = right.dataType 
match {
     case _: DayTimeIntervalType =>
-      timestampAddDayTime(start.asInstanceOf[Long], 
interval.asInstanceOf[Long], zoneId)
+      timestampAddDayTime(start.asInstanceOf[Long], 
interval.asInstanceOf[Long], zoneIdInEval)
     case CalendarIntervalType =>
       val i = interval.asInstanceOf[CalendarInterval]
-      timestampAddInterval(start.asInstanceOf[Long], i.months, i.days, 
i.microseconds, zoneId)
+      timestampAddInterval(start.asInstanceOf[Long], i.months, i.days, 
i.microseconds, zoneIdInEval)
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
+    val zid = ctx.addReferenceObj("zoneId", zoneIdInEval, 
classOf[ZoneId].getName)
     val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
     interval.dataType match {
       case _: DayTimeIntervalType =>
@@ -1749,19 +1756,22 @@ case class TimestampAddYMInterval(
 
   override def toString: String = s"$left + $right"
   override def sql: String = s"${left.sql} + ${right.sql}"
-  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, 
YearMonthIntervalType)
+  override def inputTypes: Seq[AbstractDataType] =
+    Seq(TypeCollection.AllTimestampTypes, YearMonthIntervalType)
 
-  override def dataType: DataType = TimestampType
+  override def dataType: DataType = timestamp.dataType
 
   override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
     copy(timeZoneId = Option(timeZoneId))
 
+  @transient private lazy val zoneIdInEval: ZoneId = 
zoneIdForType(left.dataType)
+
   override def nullSafeEval(micros: Any, months: Any): Any = {
-    timestampAddMonths(micros.asInstanceOf[Long], months.asInstanceOf[Int], 
zoneId)
+    timestampAddMonths(micros.asInstanceOf[Long], months.asInstanceOf[Int], 
zoneIdInEval)
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
+    val zid = ctx.addReferenceObj("zoneId", zoneIdInEval, 
classOf[ZoneId].getName)
     val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
     defineCodeGen(ctx, ev, (micros, months) => {
       s"""$dtu.timestampAddMonths($micros, $months, $zid)"""
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
index 02c95b2..f7d48fd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
@@ -88,6 +88,14 @@ private[sql] object TypeCollection {
     DayTimeIntervalType,
     YearMonthIntervalType)
 
+  /**
+   * All the supported timestamp data types
+   */
+  val AllTimestampTypes = TypeCollection(
+    TimestampType,
+    TimestampWithoutTZType
+  )
+
   def apply(types: AbstractDataType*): TypeCollection = new 
TypeCollection(types)
 
   def unapply(typ: AbstractDataType): Option[Seq[AbstractDataType]] = typ 
match {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 525123c..5719b1a 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -55,6 +55,27 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
   val time = "2013-11-08 13:10:15"
   val ts = new Timestamp(toMillis(time))
 
+  private def timestampLiteral(s: String, sdf: SimpleDateFormat, dt: 
DataType): Literal = {
+    dt match {
+      case _: TimestampType =>
+        Literal(new Timestamp(sdf.parse(s).getTime))
+
+      case _: TimestampWithoutTZType =>
+        Literal(LocalDateTime.parse(s.replace(" ", "T")))
+    }
+  }
+
+  private def timestampAnswer(s: String, sdf: SimpleDateFormat, dt: DataType): 
Any = {
+    dt match {
+      case _: TimestampType =>
+        DateTimeUtils.fromJavaTimestamp(
+          new Timestamp(sdf.parse(s).getTime))
+
+      case _: TimestampWithoutTZType =>
+        LocalDateTime.parse(s.replace(" ", "T"))
+    }
+  }
+
   test("datetime function current_date") {
     val d0 = DateTimeUtils.currentDate(UTC)
     val cd = CurrentDate(UTC_OPT).eval(EmptyRow).asInstanceOf[Int]
@@ -428,39 +449,39 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 
   test("time_add") {
     val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
-    for (zid <- outstandingZoneIds) {
-      val timeZoneId = Option(zid.getId)
-      sdf.setTimeZone(TimeZone.getTimeZone(zid))
-
-      checkEvaluation(
-        TimeAdd(
-          Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)),
-          Literal(new CalendarInterval(1, 2, 123000L)),
-          timeZoneId),
-        DateTimeUtils.fromJavaTimestamp(
-          new Timestamp(sdf.parse("2016-03-02 10:00:00.123").getTime)))
+    Seq(TimestampType, TimestampWithoutTZType).foreach { dt =>
+      for (zid <- outstandingZoneIds) {
+        val timeZoneId = Option(zid.getId)
+        sdf.setTimeZone(TimeZone.getTimeZone(zid))
+        checkEvaluation(
+          TimeAdd(
+            timestampLiteral("2016-01-29 10:00:00.000", sdf, dt),
+            Literal(new CalendarInterval(1, 2, 123000L)),
+            timeZoneId),
+          timestampAnswer("2016-03-02 10:00:00.123", sdf, dt))
 
-      checkEvaluation(
-        TimeAdd(
-          Literal.create(null, TimestampType),
-          Literal(new CalendarInterval(1, 2, 123000L)),
-          timeZoneId),
-        null)
-      checkEvaluation(
-        TimeAdd(
-          Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)),
-          Literal.create(null, CalendarIntervalType),
-          timeZoneId),
-        null)
-      checkEvaluation(
-        TimeAdd(
-          Literal.create(null, TimestampType),
-          Literal.create(null, CalendarIntervalType),
-          timeZoneId),
-        null)
-      checkConsistencyBetweenInterpretedAndCodegen(
-        (start: Expression, interval: Expression) => TimeAdd(start, interval, 
timeZoneId),
-        TimestampType, CalendarIntervalType)
+        checkEvaluation(
+          TimeAdd(
+            Literal.create(null, dt),
+            Literal(new CalendarInterval(1, 2, 123000L)),
+            timeZoneId),
+          null)
+        checkEvaluation(
+          TimeAdd(
+            timestampLiteral("2016-01-29 10:00:00.000", sdf, dt),
+            Literal.create(null, CalendarIntervalType),
+            timeZoneId),
+          null)
+        checkEvaluation(
+          TimeAdd(
+            Literal.create(null, dt),
+            Literal.create(null, CalendarIntervalType),
+            timeZoneId),
+          null)
+        checkConsistencyBetweenInterpretedAndCodegen(
+          (start: Expression, interval: Expression) => TimeAdd(start, 
interval, timeZoneId),
+          dt, CalendarIntervalType)
+      }
     }
   }
 
@@ -1600,99 +1621,99 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
     }
   }
 
-  test("SPARK-34739: add a year-month interval to a timestamp") {
+  test("SPARK-34739,SPARK-35889: add a year-month interval to a timestamp") {
     val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
-    for (zid <- outstandingZoneIds) {
-      val timeZoneId = Option(zid.getId)
-      sdf.setTimeZone(TimeZone.getTimeZone(zid))
+    Seq(TimestampType, TimestampWithoutTZType).foreach { dt =>
+      for (zid <- outstandingZoneIds) {
+        val timeZoneId = Option(zid.getId)
+        sdf.setTimeZone(TimeZone.getTimeZone(zid))
 
-      checkEvaluation(
-        TimestampAddYMInterval(
-          Literal(new Timestamp(sdf.parse("2016-01-29 10:11:12.123").getTime)),
-          Literal(Period.ofMonths(2)),
-          timeZoneId),
-        DateTimeUtils.fromJavaTimestamp(
-          new Timestamp(sdf.parse("2016-03-29 10:11:12.123").getTime)))
+        checkEvaluation(
+          TimestampAddYMInterval(
+            timestampLiteral("2016-01-29 10:11:12.123", sdf, dt),
+            Literal(Period.ofMonths(2)),
+            timeZoneId),
+          timestampAnswer("2016-03-29 10:11:12.123", sdf, dt))
 
-      checkEvaluation(
-        TimestampAddYMInterval(
-          Literal.create(null, TimestampType),
-          Literal(Period.ofMonths(1)),
-          timeZoneId),
-        null)
-      checkEvaluation(
-        TimestampAddYMInterval(
-          Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)),
-          Literal.create(null, YearMonthIntervalType()),
-          timeZoneId),
-        null)
-      checkEvaluation(
-        TimestampAddYMInterval(
-          Literal.create(null, TimestampType),
-          Literal.create(null, YearMonthIntervalType()),
-          timeZoneId),
-        null)
-      yearMonthIntervalTypes.foreach { it =>
-        checkConsistencyBetweenInterpretedAndCodegen(
-          (ts: Expression, interval: Expression) =>
-            TimestampAddYMInterval(ts, interval, timeZoneId),
-          TimestampType, it)
+        checkEvaluation(
+          TimestampAddYMInterval(
+            Literal.create(null, dt),
+            Literal(Period.ofMonths(1)),
+            timeZoneId),
+          null)
+        checkEvaluation(
+          TimestampAddYMInterval(
+            timestampLiteral("2016-01-29 10:00:00.000", sdf, dt),
+            Literal.create(null, YearMonthIntervalType()),
+            timeZoneId),
+          null)
+        checkEvaluation(
+          TimestampAddYMInterval(
+            Literal.create(null, dt),
+            Literal.create(null, YearMonthIntervalType()),
+            timeZoneId),
+          null)
+        yearMonthIntervalTypes.foreach { it =>
+          checkConsistencyBetweenInterpretedAndCodegen(
+            (ts: Expression, interval: Expression) =>
+              TimestampAddYMInterval(ts, interval, timeZoneId), dt, it)
+        }
       }
     }
   }
 
-  test("SPARK-34761: add a day-time interval to a timestamp") {
+  test("SPARK-34761,SPARK-35889: add a day-time interval to a timestamp") {
     val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
-    for (zid <- outstandingZoneIds) {
-      val timeZoneId = Option(zid.getId)
-      sdf.setTimeZone(TimeZone.getTimeZone(zid))
-      checkEvaluation(
-        TimeAdd(
-          Literal(new Timestamp(sdf.parse("2021-01-01 00:00:00.123").getTime)),
-          Literal(Duration.ofDays(10).plusMinutes(10).plusMillis(321)),
-          timeZoneId),
-        DateTimeUtils.fromJavaTimestamp(
-          new Timestamp(sdf.parse("2021-01-11 00:10:00.444").getTime)))
-      checkEvaluation(
-        TimeAdd(
-          Literal(new Timestamp(sdf.parse("2021-01-01 00:10:00.123").getTime)),
-          Literal(Duration.ofDays(-10).minusMinutes(9).minusMillis(120)),
-          timeZoneId),
-        DateTimeUtils.fromJavaTimestamp(
-          new Timestamp(sdf.parse("2020-12-22 00:01:00.003").getTime)))
+    Seq(TimestampType, TimestampWithoutTZType).foreach { dt =>
+      for (zid <- outstandingZoneIds) {
+        val timeZoneId = Option(zid.getId)
+        sdf.setTimeZone(TimeZone.getTimeZone(zid))
+        checkEvaluation(
+          TimeAdd(
+            timestampLiteral("2021-01-01 00:00:00.123", sdf, dt),
+            Literal(Duration.ofDays(10).plusMinutes(10).plusMillis(321)),
+            timeZoneId),
+          timestampAnswer("2021-01-11 00:10:00.444", sdf, dt))
+        checkEvaluation(
+          TimeAdd(
+            timestampLiteral("2021-01-01 00:10:00.123", sdf, dt),
+            Literal(Duration.ofDays(-10).minusMinutes(9).minusMillis(120)),
+            timeZoneId),
+          timestampAnswer("2020-12-22 00:01:00.003", sdf, dt))
+
+        val e = intercept[Exception] {
+          checkEvaluation(
+            TimeAdd(
+              timestampLiteral("2021-01-01 00:00:00.123", sdf, dt),
+              Literal(Duration.of(Long.MaxValue, ChronoUnit.MICROS)),
+              timeZoneId),
+            null)
+        }.getCause
+        assert(e.isInstanceOf[ArithmeticException])
+        assert(e.getMessage.contains("long overflow"))
 
-      val e = intercept[Exception] {
         checkEvaluation(
           TimeAdd(
-            Literal(new Timestamp(sdf.parse("2021-01-01 
00:00:00.123").getTime)),
-            Literal(Duration.of(Long.MaxValue, ChronoUnit.MICROS)),
+            Literal.create(null, dt),
+            Literal(Duration.ofDays(1)),
             timeZoneId),
           null)
-      }.getCause
-      assert(e.isInstanceOf[ArithmeticException])
-      assert(e.getMessage.contains("long overflow"))
-
-      checkEvaluation(
-        TimeAdd(
-          Literal.create(null, TimestampType),
-          Literal(Duration.ofDays(1)),
-          timeZoneId),
-        null)
-      checkEvaluation(
-        TimeAdd(
-          Literal(new Timestamp(sdf.parse("2021-01-01 00:00:00.123").getTime)),
-          Literal.create(null, DayTimeIntervalType()),
-          timeZoneId),
-        null)
-      checkEvaluation(
-        TimeAdd(
-          Literal.create(null, TimestampType),
-          Literal.create(null, DayTimeIntervalType()),
-          timeZoneId),
-        null)
-      dayTimeIntervalTypes.foreach { it =>
-        checkConsistencyBetweenInterpretedAndCodegen((ts: Expression, 
interval: Expression) =>
-          TimeAdd(ts, interval, timeZoneId), TimestampType, it)
+        checkEvaluation(
+          TimeAdd(
+            timestampLiteral("2021-01-01 00:00:00.123", sdf, dt),
+            Literal.create(null, DayTimeIntervalType()),
+            timeZoneId),
+          null)
+        checkEvaluation(
+          TimeAdd(
+            Literal.create(null, dt),
+            Literal.create(null, DayTimeIntervalType()),
+            timeZoneId),
+          null)
+        dayTimeIntervalTypes.foreach { it =>
+          checkConsistencyBetweenInterpretedAndCodegen((ts: Expression, 
interval: Expression) =>
+            TimeAdd(ts, interval, timeZoneId), dt, it)
+        }
       }
     }
   }
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql 
b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
index 3caaa0a..819bf4b 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
@@ -225,3 +225,14 @@ select next_day("2015-07-23", "xx");
 select next_day("xx", "Mon");
 select next_day(null, "Mon");
 select next_day(null, "xx");
+
+-- TimestampWithoutTZ + Intervals
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval 2 day;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0-0' year to month;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '1-2' year to month;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0' day to 
second;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0.1' day to 
second;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '10-9' year to month;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15' day to hour;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15:40' day to 
minute;
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 
15:40:32.99899999' day to second;
diff --git 
a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out 
b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
index 6ca0312..33d041b 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 167
+-- Number of queries: 176
 
 
 -- !query
@@ -377,7 +377,7 @@ select 1 - interval '2' second
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: 
argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7
+cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: 
argument 1 requires (timestamp or timestamp without time zone) type, however, 
'1' is of int type.; line 1 pos 7
 
 
 -- !query
@@ -1434,3 +1434,75 @@ select next_day(null, "xx")
 struct<next_day(NULL, xx):date>
 -- !query output
 NULL
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval 2 day
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '2 days':timestamp 
without time zone>
+-- !query output
+2021-06-27 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0-0' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0-0' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '1-2' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '1-2' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2022-08-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0' day to 
second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0 00:00:00' DAY TO 
SECOND:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0.1' day to 
second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0 00:00:00.1' DAY TO 
SECOND:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12.1
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '10-9' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '10-9' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2032-03-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15' day to hour
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15' DAY TO 
HOUR:timestamp without time zone>
+-- !query output
+2021-07-16 01:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15:40' day to 
minute
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15:40' DAY TO 
MINUTE:timestamp without time zone>
+-- !query output
+2021-07-16 01:51:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 
15:40:32.99899999' day to second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15:40:32.998999' 
DAY TO SECOND:timestamp without time zone>
+-- !query output
+2021-07-16 01:51:44.998999
diff --git 
a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out 
b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out
index 7f8b090..2f96b1d 100644
--- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 167
+-- Number of queries: 176
 
 
 -- !query
@@ -354,7 +354,7 @@ select 1 - interval '2' second
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: 
argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7
+cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: 
argument 1 requires (timestamp or timestamp without time zone) type, however, 
'1' is of int type.; line 1 pos 7
 
 
 -- !query
@@ -1376,3 +1376,75 @@ select next_day(null, "xx")
 struct<next_day(NULL, xx):date>
 -- !query output
 NULL
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval 2 day
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '2 days':timestamp 
without time zone>
+-- !query output
+2021-06-27 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0-0' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0-0' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '1-2' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '1-2' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2022-08-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0' day to 
second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0 00:00:00' DAY TO 
SECOND:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0.1' day to 
second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0 00:00:00.1' DAY TO 
SECOND:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12.1
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '10-9' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '10-9' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2032-03-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15' day to hour
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15' DAY TO 
HOUR:timestamp without time zone>
+-- !query output
+2021-07-16 01:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15:40' day to 
minute
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15:40' DAY TO 
MINUTE:timestamp without time zone>
+-- !query output
+2021-07-16 01:51:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 
15:40:32.99899999' day to second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15:40:32.998999' 
DAY TO SECOND:timestamp without time zone>
+-- !query output
+2021-07-16 01:51:44.998999
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out 
b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
index bfa9d7b..4ce819a 100755
--- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 167
+-- Number of queries: 176
 
 
 -- !query
@@ -354,7 +354,7 @@ select 1 - interval '2' second
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: 
argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7
+cannot resolve '1 + (- INTERVAL '2 seconds')' due to data type mismatch: 
argument 1 requires (timestamp or timestamp without time zone) type, however, 
'1' is of int type.; line 1 pos 7
 
 
 -- !query
@@ -1384,3 +1384,75 @@ select next_day(null, "xx")
 struct<next_day(NULL, xx):date>
 -- !query output
 NULL
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval 2 day
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '2 days':timestamp 
without time zone>
+-- !query output
+2021-06-27 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0-0' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0-0' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '1-2' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '1-2' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2022-08-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0' day to 
second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0 00:00:00' DAY TO 
SECOND:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '0 0:0:0.1' day to 
second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '0 00:00:00.1' DAY TO 
SECOND:timestamp without time zone>
+-- !query output
+2021-06-25 10:11:12.1
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '10-9' year to month
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '10-9' YEAR TO 
MONTH:timestamp without time zone>
+-- !query output
+2032-03-25 10:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15' day to hour
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15' DAY TO 
HOUR:timestamp without time zone>
+-- !query output
+2021-07-16 01:11:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15:40' day to 
minute
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15:40' DAY TO 
MINUTE:timestamp without time zone>
+-- !query output
+2021-07-16 01:51:12
+
+
+-- !query
+select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 
15:40:32.99899999' day to second
+-- !query schema
+struct<to_timestamp_ntz(2021-06-25 10:11:12) + INTERVAL '20 15:40:32.998999' 
DAY TO SECOND:timestamp without time zone>
+-- !query output
+2021-07-16 01:51:44.998999
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out
index 9cad1e6..771c068 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out
@@ -16,7 +16,7 @@ select cast(1 as tinyint) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is 
of tinyint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7
 
 
 -- !query
@@ -25,7 +25,7 @@ select cast(1 as smallint) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is 
of smallint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7
 
 
 -- !query
@@ -34,7 +34,7 @@ select cast(1 as int) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: 
argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; 
line 1 pos 7
+cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: 
argument 1 requires (timestamp or timestamp without time zone) type, however, 
'CAST(1 AS INT)' is of int type.; line 1 pos 7
 
 
 -- !query
@@ -43,7 +43,7 @@ select cast(1 as bigint) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is 
of bigint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7
 
 
 -- !query
@@ -52,7 +52,7 @@ select cast(1 as float) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of 
float type.; line 1 pos 7
+cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7
 
 
 -- !query
@@ -61,7 +61,7 @@ select cast(1 as double) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is 
of double type.; line 1 pos 7
+cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7
 
 
 -- !query
@@ -70,7 +70,7 @@ select cast(1 as decimal(10, 0)) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS 
DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7
+cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7
 
 
 -- !query
@@ -95,7 +95,7 @@ select cast('1' as binary) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is 
of binary type.; line 1 pos 7
+cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7
 
 
 -- !query
@@ -104,7 +104,7 @@ select cast(1 as boolean) + interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is 
of boolean type.; line 1 pos 7
+cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7
 
 
 -- !query
@@ -129,7 +129,7 @@ select interval 2 day + cast(1 as tinyint)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is 
of tinyint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7
 
 
 -- !query
@@ -138,7 +138,7 @@ select interval 2 day + cast(1 as smallint)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is 
of smallint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7
 
 
 -- !query
@@ -147,7 +147,7 @@ select interval 2 day + cast(1 as int)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: 
argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; 
line 1 pos 7
+cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: 
argument 1 requires (timestamp or timestamp without time zone) type, however, 
'CAST(1 AS INT)' is of int type.; line 1 pos 7
 
 
 -- !query
@@ -156,7 +156,7 @@ select interval 2 day + cast(1 as bigint)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is 
of bigint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7
 
 
 -- !query
@@ -165,7 +165,7 @@ select interval 2 day + cast(1 as float)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of 
float type.; line 1 pos 7
+cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7
 
 
 -- !query
@@ -174,7 +174,7 @@ select interval 2 day + cast(1 as double)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is 
of double type.; line 1 pos 7
+cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7
 
 
 -- !query
@@ -183,7 +183,7 @@ select interval 2 day + cast(1 as decimal(10, 0))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS 
DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7
+cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7
 
 
 -- !query
@@ -208,7 +208,7 @@ select interval 2 day + cast('1' as binary)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is 
of binary type.; line 1 pos 7
+cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7
 
 
 -- !query
@@ -217,7 +217,7 @@ select interval 2 day + cast(1 as boolean)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is 
of boolean type.; line 1 pos 7
+cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7
 
 
 -- !query
@@ -242,7 +242,7 @@ select cast(1 as tinyint) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS TINYINT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is 
of tinyint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS TINYINT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7
 
 
 -- !query
@@ -251,7 +251,7 @@ select cast(1 as smallint) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS SMALLINT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is 
of smallint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS SMALLINT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7
 
 
 -- !query
@@ -260,7 +260,7 @@ select cast(1 as int) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS INT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of 
int type.; line 1 pos 7
+cannot resolve 'CAST(1 AS INT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7
 
 
 -- !query
@@ -269,7 +269,7 @@ select cast(1 as bigint) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS BIGINT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is 
of bigint type.; line 1 pos 7
+cannot resolve 'CAST(1 AS BIGINT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7
 
 
 -- !query
@@ -278,7 +278,7 @@ select cast(1 as float) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS FLOAT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of 
float type.; line 1 pos 7
+cannot resolve 'CAST(1 AS FLOAT) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7
 
 
 -- !query
@@ -287,7 +287,7 @@ select cast(1 as double) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS DOUBLE) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is 
of double type.; line 1 pos 7
+cannot resolve 'CAST(1 AS DOUBLE) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7
 
 
 -- !query
@@ -296,7 +296,7 @@ select cast(1 as decimal(10, 0)) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS DECIMAL(10,0)) + (- INTERVAL '2 days')' due to data 
type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS 
DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7
+cannot resolve 'CAST(1 AS DECIMAL(10,0)) + (- INTERVAL '2 days')' due to data 
type mismatch: argument 1 requires (timestamp or timestamp without time zone) 
type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 
7
 
 
 -- !query
@@ -321,7 +321,7 @@ select cast('1' as binary) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST('1' AS BINARY) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is 
of binary type.; line 1 pos 7
+cannot resolve 'CAST('1' AS BINARY) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7
 
 
 -- !query
@@ -330,7 +330,7 @@ select cast(1 as boolean) - interval 2 day
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(1 AS BOOLEAN) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is 
of boolean type.; line 1 pos 7
+cannot resolve 'CAST(1 AS BOOLEAN) + (- INTERVAL '2 days')' due to data type 
mismatch: argument 1 requires (timestamp or timestamp without time zone) type, 
however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7
 
 
 -- !query

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to