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

maxgekk 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 2febd5c  [SPARK-35735][SQL] Take into account day-time interval fields 
in cast
2febd5c is described below

commit 2febd5c3f0c3a0c6660cfb340eb65316a1ca4acd
Author: Angerszhuuuu <angers....@gmail.com>
AuthorDate: Wed Jun 30 16:05:04 2021 +0300

    [SPARK-35735][SQL] Take into account day-time interval fields in cast
    
    ### What changes were proposed in this pull request?
    Support take into account day-time interval field in cast.
    
    ### Why are the changes needed?
    To conform to the SQL standard.
    
    ### Does this PR introduce _any_ user-facing change?
    An user can use `cast(str, DayTimeInterval(DAY, HOUR))`, for instance.
    
    ### How was this patch tested?
    Added UT.
    
    Closes #32943 from AngersZhuuuu/SPARK-35735.
    
    Authored-by: Angerszhuuuu <angers....@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../spark/sql/catalyst/util/IntervalUtils.scala    | 203 +++++++++++++++++++--
 .../sql/catalyst/expressions/CastSuiteBase.scala   | 145 ++++++++++++++-
 2 files changed, 324 insertions(+), 24 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
index 7a6de7f..30a2fa5 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
@@ -119,18 +119,27 @@ object IntervalUtils {
     }
   }
 
+  val supportedFormat = Map(
+    (YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO 
MONTH"),
+    (YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),
+    (YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH"),
+    (DT.DAY, DT.DAY) -> Seq("[+|-]d", "INTERVAL [+|-]'[+|-]d' DAY"),
+    (DT.DAY, DT.HOUR) -> Seq("[+|-]d h", "INTERVAL [+|-]'[+|-]d h' DAY TO 
HOUR"),
+    (DT.DAY, DT.MINUTE) -> Seq("[+|-]d h:m", "INTERVAL [+|-]'[+|-]d h:m' DAY 
TO MINUTE"),
+    (DT.DAY, DT.SECOND) -> Seq("[+|-]d h:m:s.n", "INTERVAL [+|-]'[+|-]d 
h:m:s.n' DAY TO SECOND"),
+    (DT.HOUR, DT.HOUR) -> Seq("[+|-]h", "INTERVAL [+|-]'[+|-]h' HOUR"),
+    (DT.HOUR, DT.MINUTE) -> Seq("[+|-]h:m", "INTERVAL [+|-]'[+|-]h:m' HOUR TO 
MINUTE"),
+    (DT.HOUR, DT.SECOND) -> Seq("[+|-]h:m:s.n", "INTERVAL [+|-]'[+|-]h:m:s.n' 
HOUR TO SECOND"),
+    (DT.MINUTE, DT.MINUTE) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MINUTE"),
+    (DT.MINUTE, DT.SECOND) -> Seq("[+|-]m:s.n", "INTERVAL [+|-]'[+|-]m:s.n' 
MINUTE TO SECOND"),
+    (DT.SECOND, DT.SECOND) -> Seq("[+|-]s.n", "INTERVAL [+|-]'[+|-]s.n' 
SECOND")
+  )
+
   def castStringToYMInterval(
       input: UTF8String,
       startField: Byte,
       endField: Byte): Int = {
 
-    val supportedFormat = Map(
-      (YM.YEAR, YM.MONTH) ->
-        Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH"),
-      (YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),
-      (YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH")
-    )
-
     def checkStringIntervalType(targetStartField: Byte, targetEndField: Byte): 
Unit = {
       if (startField != targetStartField || endField != targetEndField) {
         throw new IllegalArgumentException(s"Interval string does not match 
year-month format of " +
@@ -151,7 +160,7 @@ object IntervalUtils {
         checkStringIntervalType(YM.YEAR, YM.MONTH)
         toYMInterval(year, month, getSign(firstSign, secondSign))
       case yearMonthIndividualRegex(secondSign, value) =>
-        safeToYMInterval {
+        safeToInterval {
           val sign = getSign("+", secondSign)
           if (endField == YM.YEAR) {
             sign * Math.toIntExact(value.toLong * MONTHS_PER_YEAR)
@@ -166,7 +175,7 @@ object IntervalUtils {
           }
         }
       case yearMonthIndividualLiteralRegex(firstSign, secondSign, value, 
suffix) =>
-        safeToYMInterval {
+        safeToInterval {
           val sign = getSign(firstSign, secondSign)
           if ("YEAR".equalsIgnoreCase(suffix)) {
             checkStringIntervalType(YM.YEAR, YM.YEAR)
@@ -202,7 +211,7 @@ object IntervalUtils {
     }
   }
 
-  private def safeToYMInterval(f: => Int): Int = {
+  private def safeToInterval[T](f: => T): T = {
     try {
       f
     } catch {
@@ -213,24 +222,72 @@ object IntervalUtils {
   }
 
   private def toYMInterval(yearStr: String, monthStr: String, sign: Int): Int 
= {
-    safeToYMInterval {
+    safeToInterval {
       val years = toLongWithRange(YEAR, yearStr, 0, Integer.MAX_VALUE / 
MONTHS_PER_YEAR)
       val totalMonths = sign * (years * MONTHS_PER_YEAR + 
toLongWithRange(MONTH, monthStr, 0, 11))
       Math.toIntExact(totalMonths)
     }
   }
 
+  private val normalPattern = "(\\d{1,2})"
+  private val dayBoundPattern = "(\\d{1,9})"
+  private val hourBoundPattern = "(\\d{1,10})"
+  private val minuteBoundPattern = "(\\d{1,12})"
+  private val secondBoundPattern = "(\\d{1,13})"
+  private val microPattern = "(\\.\\d{1,9})"
+
+  private val dayHourPatternString = s"([+|-])?$dayBoundPattern $normalPattern"
+  private val dayHourRegex = (s"^$dayHourPatternString$$").r
+  private val dayHourLiteralRegex =
+    
(s"(?i)^INTERVAL\\s+([+|-])?\\'$dayHourPatternString\\'\\s+DAY\\s+TO\\s+HOUR$$").r
+
+  private val dayMinutePatternString = s"([+|-])?$dayBoundPattern 
$normalPattern:$normalPattern"
+  private val dayMinuteRegex = (s"^$dayMinutePatternString$$").r
+  private val dayMinuteLiteralRegex =
+    
(s"(?i)^INTERVAL\\s+([+|-])?\\'$dayMinutePatternString\\'\\s+DAY\\s+TO\\s+MINUTE$$").r
+
   private val daySecondPatternString =
-    "([+|-])?(\\d+) (\\d{1,2}):(\\d{1,2}):(\\d{1,2})(\\.\\d{1,9})?"
+    s"([+|-])?$dayBoundPattern 
$normalPattern:$normalPattern:$normalPattern$microPattern?"
   private val daySecondRegex = (s"^$daySecondPatternString$$").r
   private val daySecondLiteralRegex =
     
(s"(?i)^INTERVAL\\s+([+|-])?\\'$daySecondPatternString\\'\\s+DAY\\s+TO\\s+SECOND$$").r
 
+  private val hourMinutePatternString = 
s"([+|-])?$hourBoundPattern:$normalPattern"
+  private val hourMinuteRegex = (s"^$hourMinutePatternString$$").r
+  private val hourMinuteLiteralRegex =
+    
(s"(?i)^INTERVAL\\s+([+|-])?\\'$hourMinutePatternString\\'\\s+HOUR\\s+TO\\s+MINUTE$$").r
+
+  private val hourSecondPatternString =
+    s"([+|-])?$hourBoundPattern:$normalPattern:$normalPattern$microPattern?"
+  private val hourSecondRegex = (s"^$hourSecondPatternString$$").r
+  private val hourSecondLiteralRegex =
+    
(s"(?i)^INTERVAL\\s+([+|-])?\\'$hourSecondPatternString\\'\\s+HOUR\\s+TO\\s+SECOND$$").r
+
+  private val minuteSecondPatternString =
+    s"([+|-])?$minuteBoundPattern:$normalPattern$microPattern?"
+  private val minuteSecondRegex = (s"^$minuteSecondPatternString$$").r
+  private val minuteSecondLiteralRegex =
+    
(s"(?i)^INTERVAL\\s+([+|-])?\\'$minuteSecondPatternString\\'\\s+MINUTE\\s+TO\\s+SECOND$$").r
+
+  private val dayTimeIndividualPatternString = 
s"([+|-])?$secondBoundPattern(\\.\\d{1,9})?"
+  private val dayTimeIndividualRegex = 
(s"^$dayTimeIndividualPatternString$$").r
+  private val dayTimeIndividualLiteralRegex =
+    
(s"(?i)^INTERVAL\\s+([+|-])?'$dayTimeIndividualPatternString'\\s+(DAY|HOUR|MINUTE|SECOND)$$").r
+
   def castStringToDTInterval(
       input: UTF8String,
-      // TODO(SPARK-35735): Take into account day-time interval fields in cast
       startField: Byte,
       endField: Byte): Long = {
+
+    def checkStringIntervalType(targetStartField: Byte, targetEndField: Byte): 
Unit = {
+      if (startField != targetStartField || endField != targetEndField) {
+        throw new IllegalArgumentException(s"Interval string does not match 
day-time format of " +
+          s"${supportedFormat((targetStartField, targetStartField))
+            .map(format => s"`$format`").mkString(", ")} " +
+          s"when cast to ${DT(startField, endField).typeName}: 
${input.toString}")
+      }
+    }
+
     def secondAndMicro(second: String, micro: String): String = {
       if (micro != null) {
         s"$second$micro"
@@ -240,17 +297,100 @@ object IntervalUtils {
     }
 
     input.trimAll().toString match {
-      case daySecondRegex("-", day, hour, minute, second, micro) =>
-        toDTInterval(day, hour, minute, secondAndMicro(second, micro), -1)
-      case daySecondRegex(_, day, hour, minute, second, micro) =>
-        toDTInterval(day, hour, minute, secondAndMicro(second, micro), 1)
+      case dayHourRegex(sign, day, hour) =>
+        checkStringIntervalType(DT.DAY, DT.HOUR)
+        toDTInterval(day, hour, "0", "0", getSign(null, sign))
+      case dayHourLiteralRegex(firstSign, secondSign, day, hour) =>
+        checkStringIntervalType(DT.DAY, DT.HOUR)
+        toDTInterval(day, hour, "0", "0", getSign(firstSign, secondSign))
+      case dayMinuteRegex(sign, day, hour, minute) =>
+        checkStringIntervalType(DT.DAY, DT.MINUTE)
+        toDTInterval(day, hour, minute, "0", getSign(null, sign))
+      case dayMinuteLiteralRegex(firstSign, secondSign, day, hour, minute) =>
+        checkStringIntervalType(DT.DAY, DT.MINUTE)
+        toDTInterval(day, hour, minute, "0", getSign(firstSign, secondSign))
+      case daySecondRegex(sign, day, hour, minute, second, micro) =>
+        checkStringIntervalType(DT.DAY, DT.SECOND)
+        toDTInterval(day, hour, minute, secondAndMicro(second, micro), 
getSign(null, sign))
       case daySecondLiteralRegex(firstSign, secondSign, day, hour, minute, 
second, micro) =>
+        checkStringIntervalType(DT.DAY, DT.SECOND)
         toDTInterval(day, hour, minute, secondAndMicro(second, micro),
           getSign(firstSign, secondSign))
+
+      case hourMinuteRegex(sign, hour, minute) =>
+        checkStringIntervalType(DT.HOUR, DT.MINUTE)
+        toDTInterval(hour, minute, "0", getSign(null, sign))
+      case hourMinuteLiteralRegex(firstSign, secondSign, hour, minute) =>
+        checkStringIntervalType(DT.HOUR, DT.MINUTE)
+        toDTInterval(hour, minute, "0", getSign(firstSign, secondSign))
+      case hourSecondRegex(sign, hour, minute, second, micro) =>
+        checkStringIntervalType(DT.HOUR, DT.SECOND)
+        toDTInterval(hour, minute, secondAndMicro(second, micro), 
getSign(null, sign))
+      case hourSecondLiteralRegex(firstSign, secondSign, hour, minute, second, 
micro) =>
+        checkStringIntervalType(DT.HOUR, DT.SECOND)
+        toDTInterval(hour, minute, secondAndMicro(second, micro), 
getSign(firstSign, secondSign))
+
+      case minuteSecondRegex(sign, minute, second, micro) =>
+        checkStringIntervalType(DT.MINUTE, DT.SECOND)
+        toDTInterval(minute, secondAndMicro(second, micro), getSign(null, 
sign))
+      case minuteSecondLiteralRegex(firstSign, secondSign, minute, second, 
micro) =>
+        checkStringIntervalType(DT.MINUTE, DT.SECOND)
+        toDTInterval(minute, secondAndMicro(second, micro), getSign(firstSign, 
secondSign))
+
+      case dayTimeIndividualRegex(secondSign, value, suffix) =>
+        safeToInterval {
+          val sign = getSign("+", secondSign)
+          (startField, endField) match {
+            case (DT.DAY, DT.DAY) if suffix == null && value.length <= 9 =>
+              sign * value.toLong * MICROS_PER_DAY
+            case (DT.HOUR, DT.HOUR) if suffix == null && value.length <= 10 =>
+              sign * value.toLong * MICROS_PER_HOUR
+            case (DT.MINUTE, DT.MINUTE) if suffix == null && value.length <= 
12 =>
+              sign * value.toLong * MICROS_PER_MINUTE
+            case (DT.SECOND, DT.SECOND) if value.length <= 13 =>
+              sign match {
+                case 1 => parseSecondNano(secondAndMicro(value, suffix))
+                case -1 => parseSecondNano(s"-${secondAndMicro(value, 
suffix)}")
+              }
+            case (_, _) => throw new IllegalArgumentException(
+              s"Interval string does not match day-time format of " +
+                s"${supportedFormat((startField, endField))
+                    .map(format => s"`$format`").mkString(", ")} " +
+                s"when cast to ${DT(startField, endField).typeName}: 
${input.toString}")
+          }
+        }
+      case dayTimeIndividualLiteralRegex(firstSign, secondSign, value, suffix, 
unit) =>
+        safeToInterval {
+          val sign = getSign(firstSign, secondSign)
+          unit match {
+            case "DAY" if suffix == null && value.length <= 9 =>
+              checkStringIntervalType(DT.DAY, DT.DAY)
+              sign * value.toLong * MICROS_PER_DAY
+            case "HOUR" if suffix == null && value.length <= 10 =>
+              checkStringIntervalType(DT.HOUR, DT.HOUR)
+              sign * value.toLong * MICROS_PER_HOUR
+            case "MINUTE" if suffix == null && value.length <= 12 =>
+              checkStringIntervalType(DT.MINUTE, DT.MINUTE)
+              sign * value.toLong * MICROS_PER_MINUTE
+            case "SECOND" if value.length <= 13 =>
+              checkStringIntervalType(DT.SECOND, DT.SECOND)
+              sign match {
+                case 1 => parseSecondNano(secondAndMicro(value, suffix))
+                case -1 => parseSecondNano(s"-${secondAndMicro(value, 
suffix)}")
+              }
+            case _ => throw new IllegalArgumentException(
+              s"Interval string does not match day-time format of " +
+                s"${supportedFormat((startField, endField))
+                  .map(format => s"`$format`").mkString(", ")} " +
+                s"when cast to ${DT(startField, endField).typeName}: 
${input.toString}")
+          }
+        }
       case _ =>
         throw new IllegalArgumentException(
-          s"Interval string must match day-time format of `d h:m:s.n` " +
-            s"or `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND`: 
${input.toString}, " +
+          s"Interval string does not match day-time format of " +
+            s"${supportedFormat((startField, endField))
+              .map(format => s"`$format`").mkString(", ")} " +
+            s"when cast to ${DT(startField, endField).typeName}: 
${input.toString}, " +
             s"$fallbackNotice")
     }
   }
@@ -272,6 +412,31 @@ object IntervalUtils {
     micros
   }
 
+  def toDTInterval(
+      hourStr: String,
+      minuteStr: String,
+      secondStr: String,
+      sign: Int): Long = {
+    var micros = 0L
+    val hours = toLongWithRange(HOUR, hourStr, 0, 2562047788L)
+    micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR)
+    val minutes = toLongWithRange(MINUTE, minuteStr, 0, 59)
+    micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE)
+    micros = Math.addExact(micros, sign * parseSecondNano(secondStr))
+    micros
+  }
+
+  def toDTInterval(
+      minuteStr: String,
+      secondStr: String,
+      sign: Int): Long = {
+    var micros = 0L
+    val minutes = toLongWithRange(MINUTE, minuteStr, 0, 153722867280L)
+    micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE)
+    micros = Math.addExact(micros, sign * parseSecondNano(secondStr))
+    micros
+  }
+
   /**
    * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and 
[-]HH:mm:ss.nnnnnnnnn
    *
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala
index 95ea93f..48835bf 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import java.sql.{Date, Timestamp}
 import java.time.{Duration, LocalDate, LocalDateTime, Period}
 import java.time.temporal.ChronoUnit
-import java.util.{Calendar, TimeZone}
+import java.util.{Calendar, Locale, TimeZone}
 
 import scala.collection.parallel.immutable.ParVector
 
@@ -997,12 +997,12 @@ abstract class CastSuiteBase extends SparkFunSuite with 
ExpressionEvalHelper {
     }
 
     if (!isTryCast) {
-      Seq("INTERVAL '-106751991 04:00:54.775809' YEAR TO MONTH",
-        "INTERVAL '106751991 04:00:54.775808' YEAR TO MONTH").foreach { 
interval =>
-        val e = intercept[IllegalArgumentException] {
+      Seq("INTERVAL '-106751991 04:00:54.775809' DAY TO SECOND",
+        "INTERVAL '106751991 04:00:54.775808' DAY TO SECOND").foreach { 
interval =>
+        val e = intercept[ArithmeticException] {
           cast(Literal.create(interval), DayTimeIntervalType()).eval()
         }.getMessage
-        assert(e.contains("Interval string must match day-time format of"))
+        assert(e.contains("long overflow"))
       }
     }
 
@@ -1136,4 +1136,139 @@ abstract class CastSuiteBase extends SparkFunSuite with 
ExpressionEvalHelper {
         }
     }
   }
+
+  test("SPARK-35735: Take into account day-time interval fields in cast") {
+    def typeName(dataType: DayTimeIntervalType): String = {
+      if (dataType.startField == dataType.endField) {
+        
DayTimeIntervalType.fieldToString(dataType.startField).toUpperCase(Locale.ROOT)
+      } else {
+        s"${DayTimeIntervalType.fieldToString(dataType.startField)} TO " +
+          
s"${DayTimeIntervalType.fieldToString(dataType.endField)}".toUpperCase(Locale.ROOT)
+      }
+    }
+
+    Seq(("1", DayTimeIntervalType(DAY, DAY), (86400) * MICROS_PER_SECOND),
+      ("-1", DayTimeIntervalType(DAY, DAY), -(86400) * MICROS_PER_SECOND),
+      ("1 01", DayTimeIntervalType(DAY, HOUR), (86400 + 3600) * 
MICROS_PER_SECOND),
+      ("-1 01", DayTimeIntervalType(DAY, HOUR), -(86400 + 3600) * 
MICROS_PER_SECOND),
+      ("1 01:01", DayTimeIntervalType(DAY, MINUTE), (86400 + 3600 + 60) * 
MICROS_PER_SECOND),
+      ("-1 01:01", DayTimeIntervalType(DAY, MINUTE), -(86400 + 3600 + 60) * 
MICROS_PER_SECOND),
+      ("1 01:01:01.12345", DayTimeIntervalType(DAY, SECOND),
+        ((86400 + 3600 + 60 + 1.12345) * MICROS_PER_SECOND).toLong),
+      ("-1 01:01:01.12345", DayTimeIntervalType(DAY, SECOND),
+        (-(86400 + 3600 + 60 + 1.12345) * MICROS_PER_SECOND).toLong),
+
+      ("01", DayTimeIntervalType(HOUR, HOUR), (3600) * MICROS_PER_SECOND),
+      ("-01", DayTimeIntervalType(HOUR, HOUR), -(3600) * MICROS_PER_SECOND),
+      ("01:01", DayTimeIntervalType(HOUR, MINUTE), (3600 + 60) * 
MICROS_PER_SECOND),
+      ("-01:01", DayTimeIntervalType(HOUR, MINUTE), -(3600 + 60) * 
MICROS_PER_SECOND),
+      ("01:01:01.12345", DayTimeIntervalType(HOUR, SECOND),
+        ((3600 + 60 + 1.12345) * MICROS_PER_SECOND).toLong),
+      ("-01:01:01.12345", DayTimeIntervalType(HOUR, SECOND),
+        (-(3600 + 60 + 1.12345) * MICROS_PER_SECOND).toLong),
+
+      ("01", DayTimeIntervalType(MINUTE, MINUTE), (60) * MICROS_PER_SECOND),
+      ("-01", DayTimeIntervalType(MINUTE, MINUTE), -(60) * MICROS_PER_SECOND),
+      ("01:01.12345", DayTimeIntervalType(MINUTE, SECOND),
+        ((60 + 1.12345) * MICROS_PER_SECOND).toLong),
+      ("-01:01.12345", DayTimeIntervalType(MINUTE, SECOND),
+        (-(60 + 1.12345) * MICROS_PER_SECOND).toLong),
+
+      ("01.12345", DayTimeIntervalType(SECOND, SECOND), ((1.12345) * 
MICROS_PER_SECOND).toLong),
+      ("-01.12345", DayTimeIntervalType(SECOND, SECOND), (-(1.12345) * 
MICROS_PER_SECOND).toLong))
+      .foreach { case (str, dataType, dt) =>
+        checkEvaluation(cast(Literal.create(str), dataType), dt)
+        checkEvaluation(
+          cast(Literal.create(s"INTERVAL '$str' ${typeName(dataType)}"), 
dataType), dt)
+        checkEvaluation(
+          cast(Literal.create(s"INTERVAL -'$str' ${typeName(dataType)}"), 
dataType), -dt)
+      }
+
+    // Check max value
+    Seq(("INTERVAL '106751991' DAY", DayTimeIntervalType(DAY), 106751991L * 
MICROS_PER_DAY),
+      ("INTERVAL '106751991 04' DAY TO HOUR", DayTimeIntervalType(DAY, HOUR), 
9223372036800000000L),
+      ("INTERVAL '106751991 04:00' DAY TO MINUTE",
+        DayTimeIntervalType(DAY, MINUTE), 9223372036800000000L),
+      ("INTERVAL '106751991 04:00:54.775807' DAY TO SECOND", 
DayTimeIntervalType(), Long.MaxValue),
+      ("INTERVAL '2562047788' HOUR", DayTimeIntervalType(HOUR), 
9223372036800000000L),
+      ("INTERVAL '2562047788:00' HOUR TO MINUTE",
+        DayTimeIntervalType(HOUR, MINUTE), 9223372036800000000L),
+      ("INTERVAL '2562047788:00:54.775807' HOUR TO SECOND",
+        DayTimeIntervalType(HOUR, SECOND), Long.MaxValue),
+      ("INTERVAL '153722867280' MINUTE", DayTimeIntervalType(MINUTE), 
9223372036800000000L),
+      ("INTERVAL '153722867280:54.775807' MINUTE TO SECOND",
+        DayTimeIntervalType(MINUTE, SECOND), Long.MaxValue),
+      ("INTERVAL '9223372036854.775807' SECOND", DayTimeIntervalType(SECOND), 
Long.MaxValue))
+      .foreach { case (interval, dataType, dt) =>
+        checkEvaluation(cast(Literal.create(interval), dataType), dt)
+      }
+
+    Seq(("INTERVAL '-106751991' DAY", DayTimeIntervalType(DAY), -106751991L * 
MICROS_PER_DAY),
+      ("INTERVAL '-106751991 04' DAY TO HOUR",
+        DayTimeIntervalType(DAY, HOUR), -9223372036800000000L),
+      ("INTERVAL '-106751991 04:00' DAY TO MINUTE",
+        DayTimeIntervalType(DAY, MINUTE), -9223372036800000000L),
+      ("INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND", 
DayTimeIntervalType(), Long.MinValue),
+      ("INTERVAL '-2562047788' HOUR", DayTimeIntervalType(HOUR), 
-9223372036800000000L),
+      ("INTERVAL '-2562047788:00' HOUR TO MINUTE",
+        DayTimeIntervalType(HOUR, MINUTE), -9223372036800000000L),
+      ("INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND",
+        DayTimeIntervalType(HOUR, SECOND), Long.MinValue),
+      ("INTERVAL '-153722867280' MINUTE", DayTimeIntervalType(MINUTE), 
-9223372036800000000L),
+      ("INTERVAL '-153722867280:54.775808' MINUTE TO SECOND",
+        DayTimeIntervalType(MINUTE, SECOND), Long.MinValue),
+      ("INTERVAL '-9223372036854.775808' SECOND", DayTimeIntervalType(SECOND), 
Long.MinValue))
+      .foreach { case (interval, dataType, dt) =>
+        checkEvaluation(cast(Literal.create(interval), dataType), dt)
+      }
+
+    if (!isTryCast) {
+      Seq(
+        ("INTERVAL '1 01:01:01.12345' DAY TO SECOND", DayTimeIntervalType(DAY, 
HOUR)),
+        ("INTERVAL '1 01:01:01.12345' DAY TO HOUR", DayTimeIntervalType(DAY, 
SECOND)),
+        ("INTERVAL '1 01:01:01.12345' DAY TO MINUTE", DayTimeIntervalType(DAY, 
MINUTE)),
+        ("1 01:01:01.12345", DayTimeIntervalType(DAY, DAY)),
+        ("1 01:01:01.12345", DayTimeIntervalType(DAY, HOUR)),
+        ("1 01:01:01.12345", DayTimeIntervalType(DAY, MINUTE)),
+
+        ("INTERVAL '01:01:01.12345' HOUR TO SECOND", DayTimeIntervalType(DAY, 
HOUR)),
+        ("INTERVAL '01:01:01.12345' HOUR TO HOUR", DayTimeIntervalType(DAY, 
SECOND)),
+        ("INTERVAL '01:01:01.12345' HOUR TO MINUTE", DayTimeIntervalType(DAY, 
MINUTE)),
+        ("01:01:01.12345", DayTimeIntervalType(DAY, DAY)),
+        ("01:01:01.12345", DayTimeIntervalType(HOUR, HOUR)),
+        ("01:01:01.12345", DayTimeIntervalType(DAY, MINUTE)),
+        ("INTERVAL '1.23' DAY", DayTimeIntervalType(DAY)),
+        ("INTERVAL '1.23' HOUR", DayTimeIntervalType(HOUR)),
+        ("INTERVAL '1.23' MINUTE", DayTimeIntervalType(MINUTE)),
+        ("INTERVAL '1.23' SECOND", DayTimeIntervalType(MINUTE)),
+        ("1.23", DayTimeIntervalType(DAY)),
+        ("1.23", DayTimeIntervalType(HOUR)),
+        ("1.23", DayTimeIntervalType(MINUTE)),
+        ("1.23", DayTimeIntervalType(MINUTE)))
+        .foreach { case (interval, dataType) =>
+          val e = intercept[IllegalArgumentException] {
+            cast(Literal.create(interval), dataType).eval()
+          }.getMessage
+          assert(e.contains("Interval string does not match day-time format"))
+        }
+
+      // Check first field outof bound
+      Seq(("INTERVAL '1067519911' DAY", DayTimeIntervalType(DAY)),
+        ("INTERVAL '10675199111 04' DAY TO HOUR", DayTimeIntervalType(DAY, 
HOUR)),
+        ("INTERVAL '1067519911 04:00' DAY TO MINUTE", DayTimeIntervalType(DAY, 
MINUTE)),
+        ("INTERVAL '1067519911 04:00:54.775807' DAY TO SECOND", 
DayTimeIntervalType()),
+        ("INTERVAL '25620477881' HOUR", DayTimeIntervalType(HOUR)),
+        ("INTERVAL '25620477881:00' HOUR TO MINUTE", DayTimeIntervalType(HOUR, 
MINUTE)),
+        ("INTERVAL '25620477881:00:54.775807' HOUR TO SECOND", 
DayTimeIntervalType(HOUR, SECOND)),
+        ("INTERVAL '1537228672801' MINUTE", DayTimeIntervalType(MINUTE)),
+        ("INTERVAL '1537228672801:54.7757' MINUTE TO SECOND", 
DayTimeIntervalType(MINUTE, SECOND)),
+        ("INTERVAL '92233720368541.775807' SECOND", 
DayTimeIntervalType(SECOND)))
+        .foreach { case (interval, dataType) =>
+          val e = intercept[IllegalArgumentException] {
+            cast(Literal.create(interval), dataType).eval()
+          }.getMessage
+          assert(e.contains("Interval string does not match day-time format"))
+        }
+    }
+  }
 }

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

Reply via email to