You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2022/07/15 09:27:41 UTC

[spark] branch master updated: [SPARK-39787][SQL] Use error class in the parsing error of function to_timestamp

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 dbcb33e1cbd [SPARK-39787][SQL] Use error class in the parsing error of function to_timestamp
dbcb33e1cbd is described below

commit dbcb33e1cbd2719a28a9a9c09e3fe5633250606f
Author: Gengliang Wang <ge...@apache.org>
AuthorDate: Fri Jul 15 12:27:12 2022 +0300

    [SPARK-39787][SQL] Use error class in the parsing error of function to_timestamp
    
    ### What changes were proposed in this pull request?
    
    Use error class `CANNOT_PARSE_TIMESTAMP` for the parsing error of function to_timestamp
    ### Why are the changes needed?
    
    Better error message. Avoid using the java exception in the runtime errors under ANSI SQL mode.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes but minor error message improvement.
    
    ### How was this patch tested?
    
    UT
    
    Closes #37198 from gengliangwang/datetimeErrorMessage.
    
    Authored-by: Gengliang Wang <ge...@apache.org>
    Signed-off-by: Max Gekk <ma...@gmail.com>
---
 core/src/main/resources/error/error-classes.json   |  6 ++
 .../catalyst/expressions/datetimeExpressions.scala | 22 +++----
 .../spark/sql/errors/QueryExecutionErrors.scala    | 10 ++--
 .../expressions/DateExpressionsSuite.scala         | 11 ++--
 .../resources/sql-tests/results/ansi/date.sql.out  |  4 +-
 .../results/ansi/datetime-parsing-invalid.sql.out  | 68 +++++++++++-----------
 .../sql-tests/results/ansi/timestamp.sql.out       | 24 ++++----
 .../results/timestampNTZ/timestamp-ansi.sql.out    | 24 ++++----
 .../sql/errors/QueryExecutionAnsiErrorsSuite.scala | 12 ++++
 9 files changed, 96 insertions(+), 85 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index a159202d99c..3eb28bfcaf8 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -29,6 +29,12 @@
     ],
     "sqlState" : "42000"
   },
+  "CANNOT_PARSE_TIMESTAMP" : {
+    "message" : [
+      "<message>. If necessary set <ansiConfig> to \"false\" to bypass this error."
+    ],
+    "sqlState" : "42000"
+  },
   "CANNOT_UP_CAST_DATATYPE" : {
     "message" : [
       "Cannot up cast <expression> from <sourceType> to <targetType>.",
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 17241f47e03..c965d50eabf 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
@@ -1211,12 +1211,10 @@ abstract class ToTimestamp
                 formatter.parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor
               }
             } catch {
-              case e: DateTimeParseException if failOnError =>
-                throw QueryExecutionErrors.ansiDateTimeParseError(e)
               case e: DateTimeException if failOnError =>
-                throw QueryExecutionErrors.ansiDateTimeError(e)
+                throw QueryExecutionErrors.ansiDateTimeParseError(e)
               case e: ParseException if failOnError =>
-                throw QueryExecutionErrors.ansiParseError(e)
+                throw QueryExecutionErrors.ansiDateTimeParseError(e)
               case e if isParseError(e) => null
             }
           }
@@ -1226,8 +1224,8 @@ abstract class ToTimestamp
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     val javaType = CodeGenerator.javaType(dataType)
-    def parseErrorBranch(method: String): String = if (failOnError) {
-      s"throw QueryExecutionErrors.$method(e);"
+    val parseErrorBranch: String = if (failOnError) {
+      s"throw QueryExecutionErrors.ansiDateTimeParseError(e);"
     } else {
       s"${ev.isNull} = true;"
     }
@@ -1250,12 +1248,10 @@ abstract class ToTimestamp
           s"""
              |try {
              |  ${ev.value} = $formatterName.$parseMethod($datetimeStr.toString()) $downScaleCode;
-             |} catch (java.time.format.DateTimeParseException e) {
-             |  ${parseErrorBranch("ansiDateTimeParseError")}
              |} catch (java.time.DateTimeException e) {
-             |  ${parseErrorBranch("ansiDateTimeError")}
+             |  ${parseErrorBranch}
              |} catch (java.text.ParseException e) {
-             |  ${parseErrorBranch("ansiParseError")}
+             |  ${parseErrorBranch}
              |}
              |""".stripMargin)
       }.getOrElse {
@@ -1272,12 +1268,10 @@ abstract class ToTimestamp
              |  true);
              |try {
              |  ${ev.value} = $timestampFormatter.$parseMethod($string.toString()) $downScaleCode;
-             |} catch (java.time.format.DateTimeParseException e) {
-             |    ${parseErrorBranch("ansiDateTimeParseError")}
              |} catch (java.time.DateTimeException e) {
-             |    ${parseErrorBranch("ansiDateTimeError")}
+             |    ${parseErrorBranch}
              |} catch (java.text.ParseException e) {
-             |    ${parseErrorBranch("ansiParseError")}
+             |    ${parseErrorBranch}
              |}
              |""".stripMargin)
       }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index c6b19d7f771..4c030a8ac0c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -23,7 +23,6 @@ import java.net.{URISyntaxException, URL}
 import java.sql.{SQLException, SQLFeatureNotSupportedException}
 import java.text.{ParseException => JavaParseException}
 import java.time.{DateTimeException, LocalDate}
-import java.time.format.DateTimeParseException
 import java.time.temporal.ChronoField
 import java.util.ConcurrentModificationException
 import java.util.concurrent.TimeoutException
@@ -227,10 +226,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
       Array(toSQLConf(SQLConf.ANSI_ENABLED.key)))
   }
 
-  def ansiDateTimeParseError(e: DateTimeParseException): DateTimeParseException = {
-    val newMessage = s"${e.getMessage}. " +
-      s"If necessary set ${SQLConf.ANSI_ENABLED.key} to false to bypass this error."
-    new DateTimeParseException(newMessage, e.getParsedString, e.getErrorIndex, e.getCause)
+  def ansiDateTimeParseError(e: Exception): SparkDateTimeException = {
+    new SparkDateTimeException(
+      errorClass = "CANNOT_PARSE_TIMESTAMP",
+      errorSubClass = None,
+      Array(e.getMessage, toSQLConf(SQLConf.ANSI_ENABLED.key)))
   }
 
   def ansiDateTimeError(e: DateTimeException): DateTimeException = {
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 8179186d9d1..26d5cf62387 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
@@ -18,9 +18,8 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import java.sql.{Date, Timestamp}
-import java.text.{ParseException, SimpleDateFormat}
+import java.text.SimpleDateFormat
 import java.time.{DateTimeException, Duration, Instant, LocalDate, LocalDateTime, Period, ZoneId}
-import java.time.format.DateTimeParseException
 import java.time.temporal.ChronoUnit
 import java.util.{Calendar, Locale, TimeZone}
 import java.util.concurrent.TimeUnit._
@@ -29,7 +28,7 @@ import scala.language.postfixOps
 import scala.reflect.ClassTag
 import scala.util.Random
 
-import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
+import org.apache.spark.{SparkDateTimeException, SparkFunSuite, SparkUpgradeException}
 import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
 import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
 import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter}
@@ -1733,10 +1732,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
           if (!ansiEnabled) {
             exprSeq.foreach(checkEvaluation(_, null))
           } else if (policy == "LEGACY") {
-            exprSeq.foreach(checkExceptionInExpression[ParseException](_, "Unparseable"))
+            exprSeq.foreach(checkExceptionInExpression[SparkDateTimeException](_, "Unparseable"))
           } else {
             exprSeq.foreach(
-              checkExceptionInExpression[DateTimeParseException](_, "could not be parsed"))
+              checkExceptionInExpression[SparkDateTimeException](_, "could not be parsed"))
           }
 
           // LEGACY works, CORRECTED failed, EXCEPTION with SparkUpgradeException
@@ -1759,7 +1758,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
           } else {
             if (ansiEnabled) {
               exprSeq2.foreach(pair =>
-                checkExceptionInExpression[DateTimeParseException](pair._1, "could not be parsed"))
+                checkExceptionInExpression[SparkDateTimeException](pair._1, "could not be parsed"))
             } else {
               exprSeq2.foreach(pair => checkEvaluation(pair._1, null))
             }
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out
index f02c1b3053a..a6c8b273ba6 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out
@@ -136,8 +136,8 @@ select to_date("02-29", "MM-dd")
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Invalid date 'February 29' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Invalid date 'February 29' as '1970' is not a leap year. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
index b927e242168..eb93ac342ed 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
@@ -23,8 +23,8 @@ select to_timestamp('-12', 'yy')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '-12' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '-12' could not be parsed at index 0. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -62,8 +62,8 @@ select to_timestamp('366', 'D')
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Invalid date 'DayOfYear 366' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Invalid date 'DayOfYear 366' as '1970' is not a leap year. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -101,8 +101,8 @@ select to_timestamp('30-365', 'dd-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -110,8 +110,8 @@ select to_timestamp('11-365', 'MM-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -119,8 +119,8 @@ select to_timestamp('2019-366', 'yyyy-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -128,8 +128,8 @@ select to_timestamp('12-30-365', 'MM-dd-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -137,8 +137,8 @@ select to_timestamp('2020-01-365', 'yyyy-dd-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -146,8 +146,8 @@ select to_timestamp('2020-10-350', 'yyyy-MM-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -155,8 +155,8 @@ select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -174,8 +174,8 @@ select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -183,8 +183,8 @@ select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text 'Unparseable' could not be parsed at index 0. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -192,8 +192,8 @@ select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -201,8 +201,8 @@ select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text 'Unparseable' could not be parsed at index 0. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -210,8 +210,8 @@ select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -219,8 +219,8 @@ select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text 'Unparseable' could not be parsed at index 0. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -228,8 +228,8 @@ select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -237,8 +237,8 @@ select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text 'Unparseable' could not be parsed at index 0. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out
index bb730bb94ae..5ec7e51f1f2 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out
@@ -336,8 +336,8 @@ select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2019-10-06 10:11:12.' could not be parsed at index 20. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2019-10-06 10:11:12.' could not be parsed at index 20. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -401,8 +401,8 @@ select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSS
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -418,8 +418,8 @@ select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd H
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -483,8 +483,8 @@ select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '12.1232019-10-06S10:11' could not be parsed at index 7. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '12.1232019-10-06S10:11' could not be parsed at index 7. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -492,8 +492,8 @@ select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '12.1232019-10-06S10:11' could not be parsed at index 9. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '12.1232019-10-06S10:11' could not be parsed at index 9. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -565,8 +565,8 @@ select to_timestamp("02-29", "MM-dd")
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Invalid date 'February 29' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Invalid date 'February 29' as '1970' is not a leap year. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out
index 7165e753968..3f1a46927d9 100644
--- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out
@@ -337,8 +337,8 @@ select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2019-10-06 10:11:12.' could not be parsed at index 20. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2019-10-06 10:11:12.' could not be parsed at index 20. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -402,8 +402,8 @@ select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSS
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -419,8 +419,8 @@ select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd H
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -484,8 +484,8 @@ select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '12.1232019-10-06S10:11' could not be parsed at index 7. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '12.1232019-10-06S10:11' could not be parsed at index 7. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -493,8 +493,8 @@ select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
 -- !query schema
 struct<>
 -- !query output
-java.time.format.DateTimeParseException
-Text '12.1232019-10-06S10:11' could not be parsed at index 9. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Text '12.1232019-10-06S10:11' could not be parsed at index 9. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
@@ -566,8 +566,8 @@ select to_timestamp("02-29", "MM-dd")
 -- !query schema
 struct<>
 -- !query output
-java.time.DateTimeException
-Invalid date 'February 29' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+[CANNOT_PARSE_TIMESTAMP] Invalid date 'February 29' as '1970' is not a leap year. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
 
 
 -- !query
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala
index 368948ed8b3..31ee6fcde94 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala
@@ -138,4 +138,16 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest with QueryErrorsSuiteBase
           |       ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
           |""".stripMargin)
   }
+
+  test("CANNOT_PARSE_TIMESTAMP: parse string to timestamp") {
+    checkError(
+      exception = intercept[SparkDateTimeException] {
+        sql("select to_timestamp('abc', 'yyyy-MM-dd HH:mm:ss')").collect()
+      },
+      errorClass = "CANNOT_PARSE_TIMESTAMP",
+      parameters = Map(
+        "message" -> "Text 'abc' could not be parsed at index 0",
+        "ansiConfig" -> ansiConf)
+    )
+  }
 }


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