Skip to content

Commit 07e4728

Browse files
MaxGekkksbeyer
authored andcommitted
[SPARK-52693][SQL] Support +/- ANSI day-time intervals to/from TIME
### What changes were proposed in this pull request? In the PR, I propose to support the `+` and `+` operators over TIME and DAY-TIME INTERVAL. #### Syntax ``` exprA + exprB, exprB + exprA exprA - exprB ``` where - **exprA** - an expression of the TIME data type of any valid precision [0, 6]. - **exprB** - and expression of the DAY-TIME INTERVAL with any start and end fields `SECOND`, `MINUTE`, `HOUR`, `DAY`. #### Returns The result of the TIME(n) data type or raises the error `DATETIME_OVERFLOW` if the result is out of the valid range `[00:00, 24:00)`. If the result is valid, its precision `n` is the maximum precision of the input time `m` and the day-time interval `i`: `n = max(m, i)` where `i` = 6 for the end interval field `SECOND` and `0` for other fields `MINUTE`, `HOUR`, `DAY`. ### Why are the changes needed? To conform the ANSI SQL standard: <img width="867" alt="Screenshot 2025-07-07 at 09 41 49" src="https://github.com/user-attachments/assets/808a3bad-70a6-4c28-b23d-83e8399bd0e9" /> ### Does this PR introduce _any_ user-facing change? No. The TIME data type hasn't been released yet. ### How was this patch tested? By running new tests and affected test suites: ``` $ build/sbt "test:testOnly *DateTimeUtilsSuite" $ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z time.sql" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes apache#51383 from MaxGekk/time-add-interval. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Max Gekk <max.gekk@gmail.com>
1 parent 7ebf571 commit 07e4728

File tree

11 files changed

+394
-6
lines changed

11 files changed

+394
-6
lines changed

common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,4 +45,5 @@ public class DateTimeConstants {
4545
public static final long NANOS_PER_MICROS = 1000L;
4646
public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
4747
public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
48+
public static final long NANOS_PER_DAY = MICROS_PER_DAY * NANOS_PER_MICROS;
4849
}

sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -181,7 +181,7 @@ private[spark] object AnsiIntervalType extends AbstractDataType {
181181
*/
182182
private[sql] abstract class AnyTimeType extends DatetimeType
183183

184-
private[spark] object AnyTimeType extends AbstractDataType {
184+
private[spark] object AnyTimeType extends AbstractDataType with Serializable {
185185
override private[sql] def simpleString: String = "time"
186186

187187
override private[sql] def acceptsType(other: DataType): Boolean =

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/BinaryArithmeticWithDatetimeResolver.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.{
3939
Subtract,
4040
SubtractDates,
4141
SubtractTimestamps,
42+
TimeAddInterval,
4243
TimestampAddInterval,
4344
TimestampAddYMInterval,
4445
UnaryMinus
@@ -53,6 +54,7 @@ import org.apache.spark.sql.types.{
5354
StringType,
5455
TimestampNTZType,
5556
TimestampType,
57+
TimeType,
5658
YearMonthIntervalType
5759
}
5860
import org.apache.spark.sql.types.DayTimeIntervalType.DAY
@@ -80,6 +82,8 @@ object BinaryArithmeticWithDatetimeResolver {
8082
a.copy(right = Cast(a.right, a.left.dataType))
8183
case (DateType, CalendarIntervalType) =>
8284
DateAddInterval(l, r, ansiEnabled = mode == EvalMode.ANSI)
85+
case (_: TimeType, _: DayTimeIntervalType) => TimeAddInterval(l, r)
86+
case (_: DayTimeIntervalType, _: TimeType) => TimeAddInterval(r, l)
8387
case (_, CalendarIntervalType | _: DayTimeIntervalType) =>
8488
Cast(TimestampAddInterval(l, r), l.dataType)
8589
case (CalendarIntervalType, DateType) =>
@@ -118,6 +122,8 @@ object BinaryArithmeticWithDatetimeResolver {
118122
ansiEnabled = mode == EvalMode.ANSI
119123
)
120124
)
125+
case (_: TimeType, _: DayTimeIntervalType) =>
126+
TimeAddInterval(l, UnaryMinus(r, mode == EvalMode.ANSI))
121127
case (_, CalendarIntervalType | _: DayTimeIntervalType) =>
122128
Cast(DatetimeSub(l, r,
123129
TimestampAddInterval(l, UnaryMinus(r, mode == EvalMode.ANSI))), l.dataType)

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala

Lines changed: 44 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
2020
import java.time.DateTimeException
2121
import java.util.Locale
2222

23+
import org.apache.spark.SparkException
2324
import org.apache.spark.sql.catalyst.InternalRow
2425
import org.apache.spark.sql.catalyst.analysis.{ExpressionBuilder, TypeCheckResult}
2526
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess}
@@ -32,7 +33,8 @@ import org.apache.spark.sql.catalyst.util.TimeFormatter
3233
import org.apache.spark.sql.catalyst.util.TypeUtils.ordinalNumber
3334
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
3435
import org.apache.spark.sql.internal.types.StringTypeWithCollation
35-
import org.apache.spark.sql.types.{AbstractDataType, AnyTimeType, DataType, DecimalType, IntegerType, ObjectType, TimeType}
36+
import org.apache.spark.sql.types.{AbstractDataType, AnyTimeType, ByteType, DataType, DayTimeIntervalType, DecimalType, IntegerType, ObjectType, TimeType}
37+
import org.apache.spark.sql.types.DayTimeIntervalType.SECOND
3638
import org.apache.spark.unsafe.types.UTF8String
3739

3840
/**
@@ -563,3 +565,44 @@ case class MakeTime(
563565
override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): MakeTime =
564566
copy(hours = newChildren(0), minutes = newChildren(1), secsAndMicros = newChildren(2))
565567
}
568+
569+
/**
570+
* Adds day-time interval to time.
571+
*/
572+
case class TimeAddInterval(time: Expression, interval: Expression)
573+
extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes {
574+
override def nullIntolerant: Boolean = true
575+
576+
override def left: Expression = time
577+
override def right: Expression = interval
578+
579+
override def toString: String = s"$left + $right"
580+
override def sql: String = s"${left.sql} + ${right.sql}"
581+
override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType, DayTimeIntervalType)
582+
583+
override def replacement: Expression = {
584+
val (timePrecision, intervalEndField) = (time.dataType, interval.dataType) match {
585+
case (TimeType(p), DayTimeIntervalType(_, endField)) => (p, endField)
586+
case _ => throw SparkException.internalError("Unexpected input types: " +
587+
s"time type ${time.dataType.sql}, interval type ${interval.dataType.sql}.")
588+
}
589+
val intervalPrecision = if (intervalEndField < SECOND) {
590+
TimeType.MIN_PRECISION
591+
} else {
592+
TimeType.MICROS_PRECISION
593+
}
594+
val targetPrecision = Math.max(timePrecision, intervalPrecision)
595+
StaticInvoke(
596+
classOf[DateTimeUtils.type],
597+
TimeType(targetPrecision),
598+
"timeAddInterval",
599+
Seq(time, Literal(timePrecision), interval, Literal(intervalEndField),
600+
Literal(targetPrecision)),
601+
Seq(AnyTimeType, IntegerType, DayTimeIntervalType, ByteType, IntegerType),
602+
propagateNull = nullIntolerant)
603+
}
604+
605+
override protected def withNewChildrenInternal(
606+
newTime: Expression, newInterval: Expression): TimeAddInterval =
607+
copy(time = newTime, interval = newInterval)
608+
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -834,4 +834,31 @@ object DateTimeUtils extends SparkDateTimeUtils {
834834
def makeTimestampNTZ(days: Int, nanos: Long): Long = {
835835
localDateTimeToMicros(LocalDateTime.of(daysToLocalDate(days), nanosToLocalTime(nanos)))
836836
}
837+
838+
/**
839+
* Adds a day-time interval to a time.
840+
*
841+
* @param time A time in nanoseconds.
842+
* @param timePrecision The number of digits of the fraction part of time.
843+
* @param interval A day-time interval in microseconds.
844+
* @param intervalEndField The rightmost field which the interval comprises of.
845+
* Valid values: 0 (DAY), 1 (HOUR), 2 (MINUTE), 3 (SECOND).
846+
* @param targetPrecision The number of digits of the fraction part of the resulting time.
847+
* @return A time value in nanoseconds or throw an arithmetic overflow
848+
* if the result out of valid time range [00:00, 24:00).
849+
*/
850+
def timeAddInterval(
851+
time: Long,
852+
timePrecision: Int,
853+
interval: Long,
854+
intervalEndField: Byte,
855+
targetPrecision: Int): Long = {
856+
val result = MathUtils.addExact(time, MathUtils.multiplyExact(interval, NANOS_PER_MICROS))
857+
if (0 <= result && result < NANOS_PER_DAY) {
858+
truncateTimeToPrecision(result, targetPrecision)
859+
} else {
860+
throw QueryExecutionErrors.timeAddIntervalOverflowError(
861+
time, timePrecision, interval, intervalEndField)
862+
}
863+
}
837864
}

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType
4444
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
4545
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.ValueInterval
4646
import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode}
47-
import org.apache.spark.sql.catalyst.util.{sideBySide, CharsetProvider, DateTimeUtils, FailFastMode, MapData}
47+
import org.apache.spark.sql.catalyst.util.{sideBySide, CharsetProvider, DateTimeUtils, FailFastMode, IntervalUtils, MapData}
4848
import org.apache.spark.sql.connector.catalog.{CatalogNotFoundException, Table, TableProvider}
4949
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
5050
import org.apache.spark.sql.connector.expressions.Transform
@@ -2534,6 +2534,21 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
25342534
summary = "")
25352535
}
25362536

2537+
def timeAddIntervalOverflowError(
2538+
time: Long,
2539+
timePrecision: Int,
2540+
interval: Long,
2541+
intervalEndField: Byte): ArithmeticException = {
2542+
val i = toSQLValue(IntervalUtils.microsToDuration(interval),
2543+
DayTimeIntervalType(intervalEndField))
2544+
val t = toSQLValue(DateTimeUtils.nanosToLocalTime(time), TimeType(timePrecision))
2545+
new SparkArithmeticException(
2546+
errorClass = "DATETIME_OVERFLOW",
2547+
messageParameters = Map("operation" -> s"add $i to the time value $t"),
2548+
context = Array.empty,
2549+
summary = "")
2550+
}
2551+
25372552
def invalidBucketFile(path: String): Throwable = {
25382553
new SparkException(
25392554
errorClass = "INVALID_BUCKET_FILE",

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeExpressionsSuite.scala

Lines changed: 31 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,14 +17,15 @@
1717

1818
package org.apache.spark.sql.catalyst.expressions
1919

20-
import java.time.LocalTime
20+
import java.time.{Duration, LocalTime}
2121

2222
import org.apache.spark.{SPARK_DOC_ROOT, SparkDateTimeException, SparkFunSuite}
2323
import org.apache.spark.sql.AnalysisException
2424
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess}
2525
import org.apache.spark.sql.catalyst.expressions.Cast.{toSQLId, toSQLValue}
2626
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
27-
import org.apache.spark.sql.types.{Decimal, DecimalType, IntegerType, StringType, TimeType}
27+
import org.apache.spark.sql.types.{DayTimeIntervalType, Decimal, DecimalType, IntegerType, StringType, TimeType}
28+
import org.apache.spark.sql.types.DayTimeIntervalType.{DAY, HOUR, SECOND}
2829

2930
class TimeExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
3031
test("ParseToTime") {
@@ -364,4 +365,32 @@ class TimeExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
364365
(child: Expression) => SecondsOfTimeWithFraction(child).replacement,
365366
TimeType())
366367
}
368+
369+
test("Add ANSI day-time intervals to TIME") {
370+
checkEvaluation(
371+
TimeAddInterval(Literal.create(null, TimeType()), Literal(Duration.ofHours(1))),
372+
null)
373+
checkEvaluation(
374+
TimeAddInterval(Literal(LocalTime.of(12, 30)), Literal(null, DayTimeIntervalType(SECOND))),
375+
null)
376+
checkEvaluation(
377+
TimeAddInterval(Literal(LocalTime.of(8, 31)), Literal(Duration.ofMinutes(30))),
378+
LocalTime.of(8, 31).plusMinutes(30))
379+
// Maximum precision of TIME and DAY-TIME INTERVAL
380+
assert(TimeAddInterval(
381+
Literal(0L, TimeType(0)),
382+
Literal(0L, DayTimeIntervalType(DAY))).dataType == TimeType(0))
383+
assert(TimeAddInterval(
384+
Literal(1L, TimeType(TimeType.MAX_PRECISION)),
385+
Literal(1L, DayTimeIntervalType(HOUR))).dataType == TimeType(TimeType.MAX_PRECISION))
386+
assert(TimeAddInterval(
387+
Literal(2L, TimeType(TimeType.MIN_PRECISION)),
388+
Literal(2L, DayTimeIntervalType(SECOND))).dataType == TimeType(TimeType.MICROS_PRECISION))
389+
assert(TimeAddInterval(
390+
Literal(3L, TimeType(TimeType.MAX_PRECISION)),
391+
Literal(3L, DayTimeIntervalType(SECOND))).dataType == TimeType(TimeType.MAX_PRECISION))
392+
checkConsistencyBetweenInterpretedAndCodegenAllowingException(
393+
(time: Expression, interval: Expression) => TimeAddInterval(time, interval).replacement,
394+
TimeType(), DayTimeIntervalType())
395+
}
367396
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala

Lines changed: 45 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,12 +26,15 @@ import java.util.concurrent.TimeUnit
2626
import org.scalatest.matchers.must.Matchers
2727
import org.scalatest.matchers.should.Matchers._
2828

29-
import org.apache.spark.{SparkDateTimeException, SparkFunSuite, SparkIllegalArgumentException}
29+
import org.apache.spark.{SparkArithmeticException, SparkDateTimeException, SparkFunSuite, SparkIllegalArgumentException}
3030
import org.apache.spark.sql.catalyst.plans.SQLHelper
3131
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
3232
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
3333
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
3434
import org.apache.spark.sql.catalyst.util.RebaseDateTime.rebaseJulianToGregorianMicros
35+
import org.apache.spark.sql.errors.DataTypeErrors.toSQLConf
36+
import org.apache.spark.sql.internal.SqlApiConf
37+
import org.apache.spark.sql.types.DayTimeIntervalType.{HOUR, SECOND}
3538
import org.apache.spark.sql.types.Decimal
3639
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
3740

@@ -1250,4 +1253,45 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
12501253
assert(truncateTimeToPrecision(localTime(23, 59, 59, 987654), 1) ==
12511254
localTime(23, 59, 59, 900000))
12521255
}
1256+
1257+
test("add day-time interval to time") {
1258+
assert(timeAddInterval(0, 0, 0, SECOND, 6) == localTime())
1259+
assert(timeAddInterval(0, 6, MICROS_PER_DAY - 1, SECOND, 6) ==
1260+
localTime(23, 59, 59, 999999))
1261+
assert(timeAddInterval(localTime(23, 59, 59, 999999), 0, -MICROS_PER_DAY + 1, SECOND, 6) ==
1262+
localTime(0, 0))
1263+
assert(timeAddInterval(localTime(12, 30, 43, 123400), 4, 10 * MICROS_PER_MINUTE, SECOND, 6) ==
1264+
localTime(12, 40, 43, 123400))
1265+
assert(timeAddInterval(localTime(19, 31, 45, 123450), 5, 6, SECOND, 6) ==
1266+
localTime(19, 31, 45, 123456))
1267+
assert(timeAddInterval(localTime(1, 2, 3, 1), 6, MICROS_PER_HOUR, HOUR, 6) ==
1268+
localTime(2, 2, 3, 1))
1269+
1270+
checkError(
1271+
exception = intercept[SparkArithmeticException] {
1272+
timeAddInterval(1, 6, MICROS_PER_DAY, SECOND, 6)
1273+
},
1274+
condition = "DATETIME_OVERFLOW",
1275+
parameters = Map("operation" ->
1276+
"add INTERVAL '86400' SECOND to the time value TIME '00:00:00.000000001'")
1277+
)
1278+
checkError(
1279+
exception = intercept[SparkArithmeticException] {
1280+
timeAddInterval(0, 0, -1, SECOND, 6)
1281+
},
1282+
condition = "DATETIME_OVERFLOW",
1283+
parameters = Map("operation" ->
1284+
"add INTERVAL '-00.000001' SECOND to the time value TIME '00:00:00'")
1285+
)
1286+
checkError(
1287+
exception = intercept[SparkArithmeticException] {
1288+
timeAddInterval(0, 0, Long.MaxValue, SECOND, 6)
1289+
},
1290+
condition = "ARITHMETIC_OVERFLOW",
1291+
parameters = Map(
1292+
"message" -> "long overflow",
1293+
"alternative" -> "",
1294+
"config" -> toSQLConf(SqlApiConf.ANSI_ENABLED_KEY))
1295+
)
1296+
}
12531297
}

sql/core/src/test/resources/sql-tests/analyzer-results/time.sql.out

Lines changed: 101 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -420,3 +420,104 @@ SELECT cast(time'11:59:59.999999' as time without time zone)
420420
-- !query analysis
421421
Project [cast(11:59:59.999999 as time(6)) AS CAST(TIME '11:59:59.999999' AS TIME(6))#x]
422422
+- OneRowRelation
423+
424+
425+
-- !query
426+
SELECT '12:43:33.1234' :: TIME(4) + INTERVAL '01:04:05.56' HOUR TO SECOND
427+
-- !query analysis
428+
Project [cast(12:43:33.1234 as time(4)) + INTERVAL '01:04:05.56' HOUR TO SECOND AS CAST(12:43:33.1234 AS TIME(4)) + INTERVAL '01:04:05.56' HOUR TO SECOND#x]
429+
+- OneRowRelation
430+
431+
432+
-- !query
433+
SELECT TIME'08:30' + NULL
434+
-- !query analysis
435+
org.apache.spark.sql.catalyst.ExtendedAnalysisException
436+
{
437+
"errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
438+
"sqlState" : "42K09",
439+
"messageParameters" : {
440+
"left" : "\"TIME(6)\"",
441+
"right" : "\"VOID\"",
442+
"sqlExpr" : "\"(TIME '08:30:00' + NULL)\""
443+
},
444+
"queryContext" : [ {
445+
"objectType" : "",
446+
"objectName" : "",
447+
"startIndex" : 8,
448+
"stopIndex" : 25,
449+
"fragment" : "TIME'08:30' + NULL"
450+
} ]
451+
}
452+
453+
454+
-- !query
455+
SELECT TIME'00:00:00.0101' + 1
456+
-- !query analysis
457+
org.apache.spark.sql.catalyst.ExtendedAnalysisException
458+
{
459+
"errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
460+
"sqlState" : "42K09",
461+
"messageParameters" : {
462+
"left" : "\"TIME(6)\"",
463+
"right" : "\"INT\"",
464+
"sqlExpr" : "\"(TIME '00:00:00.0101' + 1)\""
465+
},
466+
"queryContext" : [ {
467+
"objectType" : "",
468+
"objectName" : "",
469+
"startIndex" : 8,
470+
"stopIndex" : 30,
471+
"fragment" : "TIME'00:00:00.0101' + 1"
472+
} ]
473+
}
474+
475+
476+
-- !query
477+
SELECT TIME'12:30' - INTERVAL '12:29:59.000001' HOUR TO SECOND
478+
-- !query analysis
479+
Project [12:30:00 + -INTERVAL '12:29:59.000001' HOUR TO SECOND AS TIME '12:30:00' + (- INTERVAL '12:29:59.000001' HOUR TO SECOND)#x]
480+
+- OneRowRelation
481+
482+
483+
-- !query
484+
SELECT '23:59:59.999999' :: TIME - INTERVAL '23:59:59.999999' HOUR TO SECOND
485+
-- !query analysis
486+
Project [cast(23:59:59.999999 as time(6)) + -INTERVAL '23:59:59.999999' HOUR TO SECOND AS CAST(23:59:59.999999 AS TIME(6)) + (- INTERVAL '23:59:59.999999' HOUR TO SECOND)#x]
487+
+- OneRowRelation
488+
489+
490+
-- !query
491+
SELECT '00:00:00.0001' :: TIME(4) - INTERVAL '0 00:00:00.0001' DAY TO SECOND
492+
-- !query analysis
493+
Project [cast(00:00:00.0001 as time(4)) + -INTERVAL '0 00:00:00.0001' DAY TO SECOND AS CAST(00:00:00.0001 AS TIME(4)) + (- INTERVAL '0 00:00:00.0001' DAY TO SECOND)#x]
494+
+- OneRowRelation
495+
496+
497+
-- !query
498+
SELECT '08:30' :: TIME(0) - INTERVAL '6' HOUR
499+
-- !query analysis
500+
Project [cast(08:30 as time(0)) + -INTERVAL '06' HOUR AS CAST(08:30 AS TIME(0)) + (- INTERVAL '06' HOUR)#x]
501+
+- OneRowRelation
502+
503+
504+
-- !query
505+
SELECT '10:00:01' :: TIME(1) - INTERVAL '1' MONTH
506+
-- !query analysis
507+
org.apache.spark.sql.catalyst.ExtendedAnalysisException
508+
{
509+
"errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
510+
"sqlState" : "42K09",
511+
"messageParameters" : {
512+
"left" : "\"TIME(1)\"",
513+
"right" : "\"INTERVAL MONTH\"",
514+
"sqlExpr" : "\"(CAST(10:00:01 AS TIME(1)) - INTERVAL '1' MONTH)\""
515+
},
516+
"queryContext" : [ {
517+
"objectType" : "",
518+
"objectName" : "",
519+
"startIndex" : 8,
520+
"stopIndex" : 49,
521+
"fragment" : "'10:00:01' :: TIME(1) - INTERVAL '1' MONTH"
522+
} ]
523+
}

0 commit comments

Comments
 (0)