You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2021/07/14 10:12:46 UTC

[spark] branch branch-3.2 updated: [SPARK-35780][SQL] Support DATE/TIMESTAMP literals across the full range

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

wenchen pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
     new c9813f7  [SPARK-35780][SQL] Support DATE/TIMESTAMP literals across the full range
c9813f7 is described below

commit c9813f74e98ea42d86cdb86c0eb6f989809a4237
Author: Linhong Liu <li...@databricks.com>
AuthorDate: Wed Jul 14 18:11:39 2021 +0800

    [SPARK-35780][SQL] Support DATE/TIMESTAMP literals across the full range
    
    ### What changes were proposed in this pull request?
    DATE/TIMESTAMP literals support years 0000 to 9999. However, internally we support a range that is much larger.
    We can add or subtract large intervals from a date/timestamp and the system will happily process and display large negative and positive dates.
    
    Since we obviously cannot put this genie back into the bottle the only thing we can do is allow matching DATE/TIMESTAMP literals.
    
    ### Why are the changes needed?
    make spark more usable and bug fix
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, after this PR, below SQL will have different results
    ```sql
    select cast('-10000-1-2' as date) as date_col
    -- before PR: NULL
    -- after PR: -10000-1-2
    ```
    
    ```sql
    select cast('2021-4294967297-11' as date) as date_col
    -- before PR: 2021-01-11
    -- after PR: NULL
    ```
    
    ### How was this patch tested?
    newly added test cases
    
    Closes #32959 from linhongliu-db/SPARK-35780-full-range-datetime.
    
    Lead-authored-by: Linhong Liu <li...@databricks.com>
    Co-authored-by: Linhong Liu <67...@users.noreply.github.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit b86645776b96b28bf34f5c208dbea1e2019f5f74)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../spark/sql/catalyst/util/DateTimeUtils.scala    | 165 +++++++++++----------
 .../catalyst/expressions/AnsiCastSuiteBase.scala   |   2 -
 .../sql/catalyst/expressions/CastSuiteBase.scala   |   6 +
 .../expressions/HashExpressionsSuite.scala         |   4 +-
 .../sql/catalyst/util/DateTimeUtilsSuite.scala     |  66 ++++++++-
 .../test/resources/sql-tests/inputs/datetime.sql   |  15 ++
 .../sql-tests/results/ansi/datetime.sql.out        |  94 +++++++++++-
 .../sql-tests/results/datetime-legacy.sql.out      |  90 ++++++++++-
 .../resources/sql-tests/results/datetime.sql.out   |  90 ++++++++++-
 .../sql-tests/results/postgreSQL/date.sql.out      |  22 +--
 .../results/timestampNTZ/datetime.sql.out          |  90 ++++++++++-
 .../org/apache/spark/sql/SQLQueryTestSuite.scala   |   2 +
 .../sql/execution/datasources/json/JsonSuite.scala |  15 +-
 .../thriftserver/ThriftServerQueryTestSuite.scala  |   3 +
 14 files changed, 548 insertions(+), 116 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 0ec6291..ae444eb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -224,12 +224,12 @@ object DateTimeUtils {
    * value. The return type is [[Option]] in order to distinguish between 0L and null. The following
    * formats are allowed:
    *
-   * `yyyy`
-   * `yyyy-[m]m`
-   * `yyyy-[m]m-[d]d`
-   * `yyyy-[m]m-[d]d `
-   * `yyyy-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   * `yyyy-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
+   * `[+-]yyyy*`
+   * `[+-]yyyy*-[m]m`
+   * `[+-]yyyy*-[m]m-[d]d`
+   * `[+-]yyyy*-[m]m-[d]d `
+   * `[+-]yyyy*-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
+   * `[+-]yyyy*-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
    * `[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
    * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
    *
@@ -249,17 +249,30 @@ object DateTimeUtils {
    *         the input string can't be parsed as timestamp, the result timestamp segments are empty.
    */
   def parseTimestampString(s: UTF8String): (Array[Int], Option[ZoneId], Boolean) = {
-    if (s == null) {
+    def isValidDigits(segment: Int, digits: Int): Boolean = {
+      // A Long is able to represent a timestamp within [+-]200 thousand years
+      val maxDigitsYear = 6
+      // For the nanosecond part, more than 6 digits is allowed, but will be truncated.
+      segment == 6 || (segment == 0 && digits >= 4 && digits <= maxDigitsYear) ||
+        (segment != 0 && segment != 6 && digits <= 2)
+    }
+    if (s == null || s.trimAll().numBytes() == 0) {
       return (Array.empty, None, false)
     }
     var tz: Option[String] = None
     val segments: Array[Int] = Array[Int](1, 1, 1, 0, 0, 0, 0, 0, 0)
     var i = 0
     var currentSegmentValue = 0
+    var currentSegmentDigits = 0
     val bytes = s.trimAll().getBytes
     var j = 0
     var digitsMilli = 0
     var justTime = false
+    var yearSign: Option[Int] = None
+    if (bytes(j) == '-' || bytes(j) == '+') {
+      yearSign = if (bytes(j) == '-') Some(-1) else Some(1)
+      j += 1
+    }
     while (j < bytes.length) {
       val b = bytes(j)
       val parsedValue = b - '0'.toByte
@@ -269,50 +282,74 @@ object DateTimeUtils {
           i += 3
         } else if (i < 2) {
           if (b == '-') {
-            if (i == 0 && j != 4) {
-              // year should have exact four digits
+            if (!isValidDigits(i, currentSegmentDigits)) {
               return (Array.empty, None, false)
             }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
-          } else if (i == 0 && b == ':') {
+          } else if (i == 0 && b == ':' && yearSign.isEmpty) {
             justTime = true
+            if (!isValidDigits(3, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(3) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i = 4
           } else {
             return (Array.empty, None, false)
           }
         } else if (i == 2) {
           if (b == ' ' || b == 'T') {
+            if (!isValidDigits(i, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
           } else {
             return (Array.empty, None, false)
           }
         } else if (i == 3 || i == 4) {
           if (b == ':') {
+            if (!isValidDigits(i, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
           } else {
             return (Array.empty, None, false)
           }
         } else if (i == 5 || i == 6) {
           if (b == '-' || b == '+') {
+            if (!isValidDigits(i, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
             tz = Some(new String(bytes, j, 1))
           } else if (b == '.' && i == 5) {
+            if (!isValidDigits(i, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
           } else {
+            if (!isValidDigits(i, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
             tz = Some(new String(bytes, j, bytes.length - j))
             j = bytes.length - 1
@@ -322,8 +359,12 @@ object DateTimeUtils {
           }
         } else {
           if (i < segments.length && (b == ':' || b == ' ')) {
+            if (!isValidDigits(i, currentSegmentDigits)) {
+              return (Array.empty, None, false)
+            }
             segments(i) = currentSegmentValue
             currentSegmentValue = 0
+            currentSegmentDigits = 0
             i += 1
           } else {
             return (Array.empty, None, false)
@@ -333,61 +374,40 @@ object DateTimeUtils {
         if (i == 6) {
           digitsMilli += 1
         }
-        currentSegmentValue = currentSegmentValue * 10 + parsedValue
+        // We will truncate the nanosecond part if there are more than 6 digits, which results
+        // in loss of precision
+        if (i != 6 || currentSegmentDigits < 6) {
+          currentSegmentValue = currentSegmentValue * 10 + parsedValue
+        }
+        currentSegmentDigits += 1
       }
       j += 1
     }
 
-    segments(i) = currentSegmentValue
-    if (!justTime && i == 0 && j != 4) {
-      // year should have exact four digits
+    if (!isValidDigits(i, currentSegmentDigits)) {
       return (Array.empty, None, false)
     }
+    segments(i) = currentSegmentValue
 
     while (digitsMilli < 6) {
       segments(6) *= 10
       digitsMilli += 1
     }
 
-    // We are truncating the nanosecond part, which results in loss of precision
-    while (digitsMilli > 6) {
-      segments(6) /= 10
-      digitsMilli -= 1
-    }
     // This step also validates time zone part
     val zoneId = tz.map {
       case "+" => ZoneOffset.ofHoursMinutes(segments(7), segments(8))
       case "-" => ZoneOffset.ofHoursMinutes(-segments(7), -segments(8))
       case zoneName: String => getZoneId(zoneName.trim)
     }
+    segments(0) *= yearSign.getOrElse(1)
     (segments, zoneId, justTime)
   }
 
   /**
    * Trims and parses a given UTF8 timestamp string to the corresponding a corresponding [[Long]]
-   * value. The return type is [[Option]] in order to distinguish between 0L and null. The following
-   * formats are allowed:
-   *
-   * `yyyy`
-   * `yyyy-[m]m`
-   * `yyyy-[m]m-[d]d`
-   * `yyyy-[m]m-[d]d `
-   * `yyyy-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   * `yyyy-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   * `[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   *
-   * where `zone_id` should have one of the forms:
-   *   - Z - Zulu time zone UTC+0
-   *   - +|-[h]h:[m]m
-   *   - A short id, see https://docs.oracle.com/javase/8/docs/api/java/time/ZoneId.html#SHORT_IDS
-   *   - An id with one of the prefixes UTC+, UTC-, GMT+, GMT-, UT+ or UT-,
-   *     and a suffix in the formats:
-   *     - +|-h[h]
-   *     - +|-hh[:]mm
-   *     - +|-hh:mm:ss
-   *     - +|-hhmmss
-   *  - Region-based zone IDs in the form `area/city`, such as `Europe/Paris`
+   * value. The return type is [[Option]] in order to distinguish between 0L and null. Please
+   * refer to `parseTimestampString` for the allowed formats
    */
   def stringToTimestamp(s: UTF8String, timeZoneId: ZoneId): Option[Long] = {
     try {
@@ -422,30 +442,8 @@ object DateTimeUtils {
    * Trims and parses a given UTF8 string to a corresponding [[Long]] value which representing the
    * number of microseconds since the epoch. The result is independent of time zones,
    * which means that zone ID in the input string will be ignored.
-   * The return type is [[Option]] in order to distinguish between 0L and null. The following
-   * formats are allowed:
-   *
-   * `yyyy`
-   * `yyyy-[m]m`
-   * `yyyy-[m]m-[d]d`
-   * `yyyy-[m]m-[d]d `
-   * `yyyy-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   * `yyyy-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
-   *
-   * where `zone_id` should have one of the forms:
-   *   - Z - Zulu time zone UTC+0
-   *   - +|-[h]h:[m]m
-   *   - A short id, see https://docs.oracle.com/javase/8/docs/api/java/time/ZoneId.html#SHORT_IDS
-   *   - An id with one of the prefixes UTC+, UTC-, GMT+, GMT-, UT+ or UT-,
-   *     and a suffix in the formats:
-   *     - +|-h[h]
-   *     - +|-hh[:]mm
-   *     - +|-hh:mm:ss
-   *     - +|-hhmmss
-   *  - Region-based zone IDs in the form `area/city`, such as `Europe/Paris`
-   *
-   * Note: The input string has to contains year/month/day fields, otherwise Spark can't determine
-   *       the value of timestamp without time zone.
+   * The return type is [[Option]] in order to distinguish between 0L and null. Please
+   * refer to `parseTimestampString` for the allowed formats.
    */
   def stringToTimestampWithoutTimeZone(s: UTF8String): Option[Long] = {
     try {
@@ -518,31 +516,42 @@ object DateTimeUtils {
    * The return type is [[Option]] in order to distinguish between 0 and null. The following
    * formats are allowed:
    *
-   * `yyyy`
-   * `yyyy-[m]m`
-   * `yyyy-[m]m-[d]d`
-   * `yyyy-[m]m-[d]d `
-   * `yyyy-[m]m-[d]d *`
-   * `yyyy-[m]m-[d]dT*`
+   * `[+-]yyyy*`
+   * `[+-]yyyy*-[m]m`
+   * `[+-]yyyy*-[m]m-[d]d`
+   * `[+-]yyyy*-[m]m-[d]d `
+   * `[+-]yyyy*-[m]m-[d]d *`
+   * `[+-]yyyy*-[m]m-[d]dT*`
    */
   def stringToDate(s: UTF8String): Option[Int] = {
-    if (s == null) {
+    def isValidDigits(segment: Int, digits: Int): Boolean = {
+      // An integer is able to represent a date within [+-]5 million years.
+      var maxDigitsYear = 7
+      (segment == 0 && digits >= 4 && digits <= maxDigitsYear) || (segment != 0 && digits <= 2)
+    }
+    if (s == null || s.trimAll().numBytes() == 0) {
       return None
     }
     val segments: Array[Int] = Array[Int](1, 1, 1)
+    var sign = 1
     var i = 0
     var currentSegmentValue = 0
+    var currentSegmentDigits = 0
     val bytes = s.trimAll().getBytes
     var j = 0
+    if (bytes(j) == '-' || bytes(j) == '+') {
+      sign = if (bytes(j) == '-') -1 else 1
+      j += 1
+    }
     while (j < bytes.length && (i < 3 && !(bytes(j) == ' ' || bytes(j) == 'T'))) {
       val b = bytes(j)
       if (i < 2 && b == '-') {
-        if (i == 0 && j != 4) {
-          // year should have exact four digits
+        if (!isValidDigits(i, currentSegmentDigits)) {
           return None
         }
         segments(i) = currentSegmentValue
         currentSegmentValue = 0
+        currentSegmentDigits = 0
         i += 1
       } else {
         val parsedValue = b - '0'.toByte
@@ -550,12 +559,12 @@ object DateTimeUtils {
           return None
         } else {
           currentSegmentValue = currentSegmentValue * 10 + parsedValue
+          currentSegmentDigits += 1
         }
       }
       j += 1
     }
-    if (i == 0 && j != 4) {
-      // year should have exact four digits
+    if (!isValidDigits(i, currentSegmentDigits)) {
       return None
     }
     if (i < 2 && j < bytes.length) {
@@ -564,7 +573,7 @@ object DateTimeUtils {
     }
     segments(i) = currentSegmentValue
     try {
-      val localDate = LocalDate.of(segments(0), segments(1), segments(2))
+      val localDate = LocalDate.of(sign * segments(0), segments(1), segments(2))
       Some(localDateToDays(localDate))
     } catch {
       case NonFatal(_) => None
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala
index a8c9dbc..ac72aa5 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala
@@ -392,8 +392,6 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
           s"Cannot cast $str to DateType.")
       }
 
-      checkCastWithParseError("12345")
-      checkCastWithParseError("12345-12-18")
       checkCastWithParseError("2015-13-18")
       checkCastWithParseError("2015-03-128")
       checkCastWithParseError("2015/03/18")
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 38e012c..dcdc6f9 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
@@ -107,6 +107,12 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
 
   test("cast string to date") {
     var c = Calendar.getInstance()
+    c.set(12345, 0, 1, 0, 0, 0)
+    c.set(Calendar.MILLISECOND, 0)
+    checkEvaluation(Cast(Literal("12345"), DateType), new Date(c.getTimeInMillis))
+    c.set(12345, 11, 18, 0, 0, 0)
+    c.set(Calendar.MILLISECOND, 0)
+    checkEvaluation(Cast(Literal("12345-12-18"), DateType), new Date(c.getTimeInMillis))
     c.set(2015, 0, 1, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
     checkEvaluation(Cast(Literal("2015"), DateType), new Date(c.getTimeInMillis))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
index aa010ae..216931e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
@@ -194,9 +194,11 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     // before epoch
     checkHiveHashForDateType("1800-01-01", -62091)
 
+    // negative year
+    checkHiveHashForDateType("-1212-01-01", -1162202)
+
     // Invalid input: bad date string. Hive returns 0 for such cases
     intercept[NoSuchElementException](checkHiveHashForDateType("0-0-0", 0))
-    intercept[NoSuchElementException](checkHiveHashForDateType("-1212-01-01", 0))
     intercept[NoSuchElementException](checkHiveHashForDateType("2016-99-99", 0))
 
     // Invalid input: Empty string. Hive returns 0 for this case
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
index e6bf893..2b7b941 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
@@ -142,13 +142,31 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
     assert(toDate("2015.03.18").isEmpty)
     assert(toDate("20150318").isEmpty)
     assert(toDate("2015-031-8").isEmpty)
-    assert(toDate("02015-03-18").isEmpty)
     assert(toDate("015-03-18").isEmpty)
     assert(toDate("015").isEmpty)
-    assert(toDate("02015").isEmpty)
     assert(toDate("1999 08 01").isEmpty)
     assert(toDate("1999-08 01").isEmpty)
     assert(toDate("1999 08").isEmpty)
+    assert(toDate("").isEmpty)
+    assert(toDate("   ").isEmpty)
+  }
+
+  test("SPARK-35780: support full range of date string") {
+    assert(toDate("02015-03-18").get === days(2015, 3, 18))
+    assert(toDate("02015").get === days(2015, 1, 1))
+    assert(toDate("-02015").get === days(-2015, 1, 1))
+    assert(toDate("999999-1-28").get === days(999999, 1, 28))
+    assert(toDate("-999999-1-28").get === days(-999999, 1, 28))
+    assert(toDate("0001-1-28").get === days(1, 1, 28))
+    // Int.MaxValue and Int.MaxValue + 1 day
+    assert(toDate("5881580-7-11").get === days(5881580, 7, 11))
+    assert(toDate("5881580-7-12").isEmpty)
+    // Int.MinValue and Int.MinValue - 1 day
+    assert(toDate("-5877641-6-23").get === days(-5877641, 6, 23))
+    assert(toDate("-5877641-6-22").isEmpty)
+    // Check overflow of single segment in date format
+    assert(toDate("4294967297").isEmpty)
+    assert(toDate("2021-4294967297-11").isEmpty)
   }
 
   private def toTimestamp(str: String, zoneId: ZoneId): Option[Long] = {
@@ -254,7 +272,6 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
       checkStringToTimestamp("2011-05-06 07:08:09.1000", expected)
 
       checkStringToTimestamp("238", None)
-      checkStringToTimestamp("00238", None)
       checkStringToTimestamp("2015-03-18 123142", None)
       checkStringToTimestamp("2015-03-18T123123", None)
       checkStringToTimestamp("2015-03-18X", None)
@@ -262,7 +279,6 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
       checkStringToTimestamp("2015.03.18", None)
       checkStringToTimestamp("20150318", None)
       checkStringToTimestamp("2015-031-8", None)
-      checkStringToTimestamp("02015-01-18", None)
       checkStringToTimestamp("015-01-18", None)
       checkStringToTimestamp("2015-03-18T12:03.17-20:0", None)
       checkStringToTimestamp("2015-03-18T12:03.17-0:70", None)
@@ -270,6 +286,9 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
       checkStringToTimestamp("1999 08 01", None)
       checkStringToTimestamp("1999-08 01", None)
       checkStringToTimestamp("1999 08", None)
+      checkStringToTimestamp("", None)
+      checkStringToTimestamp("    ", None)
+      checkStringToTimestamp("+", None)
 
       // Truncating the fractional seconds
       expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, zid = UTC))
@@ -283,6 +302,45 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
     }
   }
 
+  test("SPARK-35780: support full range of timestamp string") {
+    def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = {
+      assert(toTimestamp(str, UTC) === expected)
+    }
+
+    checkStringToTimestamp("-1969-12-31 16:00:00", Option(date(-1969, 12, 31, 16, zid = UTC)))
+    checkStringToTimestamp("02015-03-18 16:00:00", Option(date(2015, 3, 18, 16, zid = UTC)))
+    checkStringToTimestamp("000001", Option(date(1, 1, 1, 0, zid = UTC)))
+    checkStringToTimestamp("-000001", Option(date(-1, 1, 1, 0, zid = UTC)))
+    checkStringToTimestamp("00238", Option(date(238, 1, 1, 0, zid = UTC)))
+    checkStringToTimestamp("99999-03-01T12:03:17", Option(date(99999, 3, 1, 12, 3, 17, zid = UTC)))
+    checkStringToTimestamp("+12:12:12", None)
+    checkStringToTimestamp("-12:12:12", None)
+    checkStringToTimestamp("", None)
+    checkStringToTimestamp("    ", None)
+    checkStringToTimestamp("+", None)
+    // Long.MaxValue and Long.MaxValue + 1 micro seconds
+    checkStringToTimestamp(
+      "294247-01-10T04:00:54.775807Z",
+      Option(date(294247, 1, 10, 4, 0, 54, 775807, zid = UTC)))
+    checkStringToTimestamp("294247-01-10T04:00:54.775808Z", None)
+    // Long.MinValue and Long.MinValue - 1 micro seconds
+    checkStringToTimestamp(
+      "-290308-12-21T19:59:05.224192Z",
+      Option(date(-290308, 12, 21, 19, 59, 5, 224192, zid = UTC)))
+    // Check overflow of single segment in timestamp format
+    checkStringToTimestamp("-290308-12-21T19:59:05.224191Z", None)
+    checkStringToTimestamp("4294967297", None)
+    checkStringToTimestamp("2021-4294967297-11", None)
+    checkStringToTimestamp("4294967297:30:00", None)
+    checkStringToTimestamp("2021-11-4294967297T12:30:00", None)
+    checkStringToTimestamp("2021-01-01T12:4294967297:00", None)
+    checkStringToTimestamp("2021-01-01T12:30:4294967297", None)
+    checkStringToTimestamp("2021-01-01T12:30:4294967297.123456", None)
+    checkStringToTimestamp("2021-01-01T12:30:4294967297+07:30", None)
+    checkStringToTimestamp("2021-01-01T12:30:4294967297UTC", None)
+    checkStringToTimestamp("2021-01-01T12:30:4294967297+4294967297:30", None)
+  }
+
   test("SPARK-15379: special invalid date string") {
     // Test stringToDate
     assert(toDate("2015-02-29 00:00:00").isEmpty)
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 74a451e..2c5a1aa 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
@@ -278,3 +278,18 @@ SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 60.007);
 SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678);
 SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678, 'CET');
 SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007);
+
+-- datetime with year outside [0000-9999]
+select date'999999-03-18';
+select date'-0001-1-28';
+select date'0015';
+select cast('015' as date);
+select cast('2021-4294967297-11' as date);
+
+select timestamp'-1969-12-31 16:00:00';
+select timestamp'0015-03-18 16:00:00';
+select timestamp'-000001';
+select timestamp'99999-03-18T12:03:17';
+select cast('4294967297' as timestamp);
+select cast('2021-01-01T12:30:4294967297.123456' as timestamp);
+
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 4e999f3..99b8d11 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: 206
+-- Number of queries: 217
 
 
 -- !query
@@ -1748,3 +1748,95 @@ struct<>
 -- !query output
 java.time.DateTimeException
 The fraction of sec must be zero. Valid range is [0, 60].
+
+
+-- !query
+select date'999999-03-18'
+-- !query schema
+struct<DATE '+999999-03-18':date>
+-- !query output
++999999-03-18
+
+
+-- !query
+select date'-0001-1-28'
+-- !query schema
+struct<DATE '-0001-01-28':date>
+-- !query output
+-0001-01-28
+
+
+-- !query
+select date'0015'
+-- !query schema
+struct<DATE '0015-01-01':date>
+-- !query output
+0015-01-01
+
+
+-- !query
+select cast('015' as date)
+-- !query schema
+struct<>
+-- !query output
+java.time.DateTimeException
+Cannot cast 015 to DateType.
+
+
+-- !query
+select cast('2021-4294967297-11' as date)
+-- !query schema
+struct<>
+-- !query output
+java.time.DateTimeException
+Cannot cast 2021-4294967297-11 to DateType.
+
+
+-- !query
+select timestamp'-1969-12-31 16:00:00'
+-- !query schema
+struct<TIMESTAMP '-1969-12-31 16:00:00':timestamp>
+-- !query output
+-1969-12-31 16:00:00
+
+
+-- !query
+select timestamp'0015-03-18 16:00:00'
+-- !query schema
+struct<TIMESTAMP '0015-03-18 16:00:00':timestamp>
+-- !query output
+0015-03-18 16:00:00
+
+
+-- !query
+select timestamp'-000001'
+-- !query schema
+struct<TIMESTAMP '-0001-01-01 00:00:00':timestamp>
+-- !query output
+-0001-01-01 00:00:00
+
+
+-- !query
+select timestamp'99999-03-18T12:03:17'
+-- !query schema
+struct<TIMESTAMP '+99999-03-18 12:03:17':timestamp>
+-- !query output
++99999-03-18 12:03:17
+
+
+-- !query
+select cast('4294967297' as timestamp)
+-- !query schema
+struct<>
+-- !query output
+java.time.DateTimeException
+Cannot cast 4294967297 to TimestampType.
+
+
+-- !query
+select cast('2021-01-01T12:30:4294967297.123456' as timestamp)
+-- !query schema
+struct<>
+-- !query output
+java.time.DateTimeException
+Cannot cast 2021-01-01T12:30:4294967297.123456 to TimestampType.
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 2108808..6248781 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: 206
+-- Number of queries: 217
 
 
 -- !query
@@ -1689,3 +1689,91 @@ SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007)
 struct<make_timestamp_ltz(2021, 7, 11, 6, 30, 60.007):timestamp>
 -- !query output
 NULL
+
+
+-- !query
+select date'999999-03-18'
+-- !query schema
+struct<DATE '999999-03-18':date>
+-- !query output
+999999-03-18
+
+
+-- !query
+select date'-0001-1-28'
+-- !query schema
+struct<DATE '0002-01-28':date>
+-- !query output
+0002-01-28
+
+
+-- !query
+select date'0015'
+-- !query schema
+struct<DATE '0015-01-01':date>
+-- !query output
+0015-01-01
+
+
+-- !query
+select cast('015' as date)
+-- !query schema
+struct<CAST(015 AS DATE):date>
+-- !query output
+NULL
+
+
+-- !query
+select cast('2021-4294967297-11' as date)
+-- !query schema
+struct<CAST(2021-4294967297-11 AS DATE):date>
+-- !query output
+NULL
+
+
+-- !query
+select timestamp'-1969-12-31 16:00:00'
+-- !query schema
+struct<TIMESTAMP '-1969-12-31 16:00:00':timestamp>
+-- !query output
+-1969-12-31 16:00:00
+
+
+-- !query
+select timestamp'0015-03-18 16:00:00'
+-- !query schema
+struct<TIMESTAMP '0015-03-18 16:00:00':timestamp>
+-- !query output
+0015-03-18 16:00:00
+
+
+-- !query
+select timestamp'-000001'
+-- !query schema
+struct<TIMESTAMP '-0001-01-01 00:00:00':timestamp>
+-- !query output
+-0001-01-01 00:00:00
+
+
+-- !query
+select timestamp'99999-03-18T12:03:17'
+-- !query schema
+struct<TIMESTAMP '+99999-03-18 12:03:17':timestamp>
+-- !query output
++99999-03-18 12:03:17
+
+
+-- !query
+select cast('4294967297' as timestamp)
+-- !query schema
+struct<CAST(4294967297 AS TIMESTAMP):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select cast('2021-01-01T12:30:4294967297.123456' as timestamp)
+-- !query schema
+struct<CAST(2021-01-01T12:30:4294967297.123456 AS TIMESTAMP):timestamp>
+-- !query output
+NULL
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 f6278f6..5882a58 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: 206
+-- Number of queries: 217
 
 
 -- !query
@@ -1697,3 +1697,91 @@ SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007)
 struct<make_timestamp_ltz(2021, 7, 11, 6, 30, 60.007):timestamp>
 -- !query output
 NULL
+
+
+-- !query
+select date'999999-03-18'
+-- !query schema
+struct<DATE '+999999-03-18':date>
+-- !query output
++999999-03-18
+
+
+-- !query
+select date'-0001-1-28'
+-- !query schema
+struct<DATE '-0001-01-28':date>
+-- !query output
+-0001-01-28
+
+
+-- !query
+select date'0015'
+-- !query schema
+struct<DATE '0015-01-01':date>
+-- !query output
+0015-01-01
+
+
+-- !query
+select cast('015' as date)
+-- !query schema
+struct<CAST(015 AS DATE):date>
+-- !query output
+NULL
+
+
+-- !query
+select cast('2021-4294967297-11' as date)
+-- !query schema
+struct<CAST(2021-4294967297-11 AS DATE):date>
+-- !query output
+NULL
+
+
+-- !query
+select timestamp'-1969-12-31 16:00:00'
+-- !query schema
+struct<TIMESTAMP '-1969-12-31 16:00:00':timestamp>
+-- !query output
+-1969-12-31 16:00:00
+
+
+-- !query
+select timestamp'0015-03-18 16:00:00'
+-- !query schema
+struct<TIMESTAMP '0015-03-18 16:00:00':timestamp>
+-- !query output
+0015-03-18 16:00:00
+
+
+-- !query
+select timestamp'-000001'
+-- !query schema
+struct<TIMESTAMP '-0001-01-01 00:00:00':timestamp>
+-- !query output
+-0001-01-01 00:00:00
+
+
+-- !query
+select timestamp'99999-03-18T12:03:17'
+-- !query schema
+struct<TIMESTAMP '+99999-03-18 12:03:17':timestamp>
+-- !query output
++99999-03-18 12:03:17
+
+
+-- !query
+select cast('4294967297' as timestamp)
+-- !query schema
+struct<CAST(4294967297 AS TIMESTAMP):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select cast('2021-01-01T12:30:4294967297.123456' as timestamp)
+-- !query schema
+struct<CAST(2021-01-01T12:30:4294967297.123456 AS TIMESTAMP):timestamp>
+-- !query output
+NULL
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out
index a959284..8eee6a7 100755
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out
@@ -454,29 +454,17 @@ struct<DATE '4714-11-23':date>
 -- !query
 SELECT date '5874897-12-31'
 -- !query schema
-struct<>
+struct<DATE '+5874897-12-31':date>
 -- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 5874897-12-31(line 1, pos 7)
-
-== SQL ==
-SELECT date '5874897-12-31'
--------^^^
++5874897-12-31
 
 
 -- !query
 SELECT date '5874898-01-01'
 -- !query schema
-struct<>
+struct<DATE '+5874898-01-01':date>
 -- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 5874898-01-01(line 1, pos 7)
-
-== SQL ==
-SELECT date '5874898-01-01'
--------^^^
++5874898-01-01
 
 
 -- !query
@@ -584,7 +572,7 @@ select make_date(-44, 3, 15)
 -- !query schema
 struct<make_date(-44, 3, 15):date>
 -- !query output
-0045-03-15
+-0044-03-15
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out
index 69c4b8f..afccbcc 100644
--- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/datetime.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 206
+-- Number of queries: 217
 
 
 -- !query
@@ -1698,3 +1698,91 @@ SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007)
 struct<make_timestamp_ltz(2021, 7, 11, 6, 30, 60.007):timestamp>
 -- !query output
 NULL
+
+
+-- !query
+select date'999999-03-18'
+-- !query schema
+struct<DATE '+999999-03-18':date>
+-- !query output
++999999-03-18
+
+
+-- !query
+select date'-0001-1-28'
+-- !query schema
+struct<DATE '-0001-01-28':date>
+-- !query output
+-0001-01-28
+
+
+-- !query
+select date'0015'
+-- !query schema
+struct<DATE '0015-01-01':date>
+-- !query output
+0015-01-01
+
+
+-- !query
+select cast('015' as date)
+-- !query schema
+struct<CAST(015 AS DATE):date>
+-- !query output
+NULL
+
+
+-- !query
+select cast('2021-4294967297-11' as date)
+-- !query schema
+struct<CAST(2021-4294967297-11 AS DATE):date>
+-- !query output
+NULL
+
+
+-- !query
+select timestamp'-1969-12-31 16:00:00'
+-- !query schema
+struct<TIMESTAMP_NTZ '-1969-12-31 16:00:00':timestamp_ntz>
+-- !query output
+-1969-12-31 16:00:00
+
+
+-- !query
+select timestamp'0015-03-18 16:00:00'
+-- !query schema
+struct<TIMESTAMP_NTZ '0015-03-18 16:00:00':timestamp_ntz>
+-- !query output
+0015-03-18 16:00:00
+
+
+-- !query
+select timestamp'-000001'
+-- !query schema
+struct<TIMESTAMP_NTZ '-0001-01-01 00:00:00':timestamp_ntz>
+-- !query output
+-0001-01-01 00:00:00
+
+
+-- !query
+select timestamp'99999-03-18T12:03:17'
+-- !query schema
+struct<TIMESTAMP_NTZ '+99999-03-18 12:03:17':timestamp_ntz>
+-- !query output
++99999-03-18 12:03:17
+
+
+-- !query
+select cast('4294967297' as timestamp)
+-- !query schema
+struct<CAST(4294967297 AS TIMESTAMP_NTZ):timestamp_ntz>
+-- !query output
+NULL
+
+
+-- !query
+select cast('2021-01-01T12:30:4294967297.123456' as timestamp)
+-- !query schema
+struct<CAST(2021-01-01T12:30:4294967297.123456 AS TIMESTAMP_NTZ):timestamp_ntz>
+-- !query output
+NULL
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
index d194293..d5a34ae 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
@@ -143,6 +143,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper
   protected override def sparkConf: SparkConf = super.sparkConf
     // Fewer shuffle partitions to speed up testing.
     .set(SQLConf.SHUFFLE_PARTITIONS, 4)
+    // use Java 8 time API to handle negative years properly
+    .set(SQLConf.DATETIME_JAVA8API_ENABLED, true)
 
   // SPARK-32106 Since we add SQL test 'transform.sql' will use `cat` command,
   // here we need to ignore it.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
index b707a48..dab1255 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
@@ -21,7 +21,7 @@ import java.io._
 import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException}
 import java.nio.file.Files
 import java.sql.{Date, Timestamp}
-import java.time.{LocalDate, ZoneId}
+import java.time.ZoneId
 import java.util.Locale
 
 import com.fasterxml.jackson.core.JsonFactory
@@ -1510,8 +1510,7 @@ abstract class JsonSuite
       """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
       """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
       """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
-      """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" ::
-      """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil
+      """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil
     // scalastyle:on
 
     // Generate data for the current version.
@@ -1538,7 +1537,6 @@ abstract class JsonSuite
           "Spark 1.4.1",
           "Spark 1.4.1",
           "Spark 1.5.0",
-          "Spark 1.5.0",
           "Spark " + spark.sparkContext.version,
           "Spark " + spark.sparkContext.version)
       val expectedResult = col0Values.map { v =>
@@ -2684,16 +2682,13 @@ abstract class JsonSuite
   }
 
   test("SPARK-30960, SPARK-31641: parse date/timestamp string with legacy format") {
-    val julianDay = -141704 // 1582-01-01 in Julian calendar
     val ds = Seq(
-      s"{'t': '2020-1-12 3:23:34.12', 'd': '2020-1-12 T', 'd2': '12345', 'd3': '$julianDay'}"
+      s"{'t': '2020-1-12 3:23:34.12', 'd': '2020-1-12 T'}"
     ).toDS()
-    val json = spark.read.schema("t timestamp, d date, d2 date, d3 date").json(ds)
+    val json = spark.read.schema("t timestamp, d date").json(ds)
     checkAnswer(json, Row(
       Timestamp.valueOf("2020-1-12 3:23:34.12"),
-      Date.valueOf("2020-1-12"),
-      Date.valueOf(LocalDate.ofEpochDay(12345)),
-      Date.valueOf("1582-01-01")))
+      Date.valueOf("2020-1-12")))
   }
 
   test("exception mode for parsing date/timestamp string") {
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala
index f405461..6df6abb 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala
@@ -83,6 +83,9 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ
     "postgreSQL/case.sql",
     // SPARK-28624
     "date.sql",
+    "datetime.sql",
+    "datetime-legacy.sql",
+    "ansi/datetime.sql",
     // SPARK-28620
     "postgreSQL/float4.sql",
     // SPARK-28636

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