diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 34588fae5a45c..0061236483452 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -181,6 +181,17 @@ "UNSUPPORTED_OPERATION" : { "message" : [ "The operation is not supported: " ] }, + "UNSUPPORTED_SAVE_MODE" : { + "message" : [ "The save mode is not supported for:" ], + "subClass" : { + "EXISTENT_PATH" : { + "message" : [ "an existent path." ] + }, + "NON_EXISTENT_PATH" : { + "message" : [ "a not existent path." ] + } + } + }, "WRITING_JOB_ABORTED" : { "message" : [ "Writing job aborted" ], "sqlState" : "40000" diff --git a/core/src/main/scala/org/apache/spark/ErrorInfo.scala b/core/src/main/scala/org/apache/spark/ErrorInfo.scala index 99e081ebb45ea..613c591eec2c7 100644 --- a/core/src/main/scala/org/apache/spark/ErrorInfo.scala +++ b/core/src/main/scala/org/apache/spark/ErrorInfo.scala @@ -28,14 +28,30 @@ import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.apache.spark.util.Utils +/** + * Information associated with an error subclass. + * + * @param subClass SubClass associated with this class. + * @param message C-style message format compatible with printf. + * The error message is constructed by concatenating the lines with newlines. + */ +private[spark] case class ErrorSubInfo(message: Seq[String]) { + // For compatibility with multi-line error messages + @JsonIgnore + val messageFormat: String = message.mkString("\n") +} + /** * Information associated with an error class. * * @param sqlState SQLSTATE associated with this class. + * @param subClass A sequence of subclasses * @param message C-style message format compatible with printf. * The error message is constructed by concatenating the lines with newlines. */ -private[spark] case class ErrorInfo(message: Seq[String], sqlState: Option[String]) { +private[spark] case class ErrorInfo(message: Seq[String], + subClass: Option[Map[String, ErrorSubInfo]], + sqlState: Option[String]) { // For compatibility with multi-line error messages @JsonIgnore val messageFormat: String = message.mkString("\n") @@ -61,13 +77,25 @@ private[spark] object SparkThrowableHelper { queryContext: String = ""): String = { val errorInfo = errorClassToInfoMap.getOrElse(errorClass, throw new IllegalArgumentException(s"Cannot find error class '$errorClass'")) + val (displayClass, displayMessageParameters, displayFormat) = if (errorInfo.subClass.isEmpty) { + (errorClass, messageParameters, errorInfo.messageFormat) + } else { + val subClass = errorInfo.subClass.get + val subErrorClass = messageParameters.head + val errorSubInfo = subClass.getOrElse(subErrorClass, + throw new IllegalArgumentException(s"Cannot find sub error class '$subErrorClass'")) + (errorClass + "." + subErrorClass, messageParameters.tail, + errorInfo.messageFormat + " " + errorSubInfo.messageFormat) + } + val displayMessage = String.format( + displayFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), + displayMessageParameters : _*) val displayQueryContext = if (queryContext.isEmpty) { "" } else { s"\n$queryContext" } - String.format(errorInfo.messageFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), - messageParameters: _*) + displayQueryContext + s"[$displayClass] $displayMessage$displayQueryContext" } def getSqlState(errorClass: String): String = { diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 8b43f07675c14..9c6465732c1da 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -125,14 +125,14 @@ class SparkThrowableSuite extends SparkFunSuite { // Does not fail with too many args (expects 0 args) assert(getMessage("DIVIDE_BY_ZERO", Array("foo", "bar", "baz")) == - "Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. " + - "If necessary set foo " + - "to \"false\" (except for ANSI interval type) to bypass this error.") + "[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and " + + "return NULL instead. If necessary set foo " + + "to \"false\" (except for ANSI interval type) to bypass this error.") } test("Error message is formatted") { assert(getMessage("MISSING_COLUMN", Array("foo", "bar, baz")) == - "Column 'foo' does not exist. Did you mean one of the following? [bar, baz]") + "[MISSING_COLUMN] Column 'foo' does not exist. Did you mean one of the following? [bar, baz]") } test("Try catching legacy SparkError") { diff --git a/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala b/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala index 3a6e7f4c12472..cf35129aca662 100644 --- a/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala @@ -89,7 +89,8 @@ class GraphiteSinkSuite extends SparkFunSuite { new GraphiteSink(props, registry) } assert(e.getErrorClass === "GRAPHITE_SINK_PROPERTY_MISSING") - assert(e.getMessage === "Graphite sink requires 'host' property.") + assert(e.getMessage === "[GRAPHITE_SINK_PROPERTY_MISSING] " + + "Graphite sink requires 'host' property.") } test("GraphiteSink without port") { @@ -101,7 +102,8 @@ class GraphiteSinkSuite extends SparkFunSuite { new GraphiteSink(props, registry) } assert(e.getErrorClass === "GRAPHITE_SINK_PROPERTY_MISSING") - assert(e.getMessage === "Graphite sink requires 'port' property.") + assert(e.getMessage === "[GRAPHITE_SINK_PROPERTY_MISSING] " + + "Graphite sink requires 'port' property.") } test("GraphiteSink with invalid protocol") { @@ -115,6 +117,6 @@ class GraphiteSinkSuite extends SparkFunSuite { new GraphiteSink(props, registry) } assert(e.getErrorClass === "GRAPHITE_SINK_INVALID_PROTOCOL") - assert(e.getMessage === "Invalid Graphite protocol: http") + assert(e.getMessage === "[GRAPHITE_SINK_INVALID_PROTOCOL] Invalid Graphite protocol: http") } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index b33708d24e7cb..d4e5eb224c2d9 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -110,7 +110,8 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi val e = intercept[SparkOutOfMemoryError] { sorter.insertRecord(bytes, Platform.BYTE_ARRAY_OFFSET, 1, 0) } - assert(e.getMessage == "Unable to acquire 800 bytes of memory, got 400") + assert(e.getMessage == "[UNABLE_TO_ACQUIRE_MEMORY] " + + "Unable to acquire 800 bytes of memory, got 400") assert(e.getErrorClass == "UNABLE_TO_ACQUIRE_MEMORY") assert(e.getSqlState == null) } 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 21fe0b9267014..b661c6a0b7ac2 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 @@ -652,8 +652,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { """.stripMargin) } - def unsupportedSaveModeError(saveMode: String, pathExists: Boolean): Throwable = { - new IllegalStateException(s"unsupported save mode $saveMode ($pathExists)") + def saveModeUnsupportedError(saveMode: Any, pathExists: Boolean): Throwable = { + pathExists match { + case true => new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_SAVE_MODE", + messageParameters = Array("EXISTENT_PATH", toSQLValue(saveMode, StringType))) + case _ => new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_SAVE_MODE", + messageParameters = Array("NON_EXISTENT_PATH", toSQLValue(saveMode, StringType))) + } } def cannotClearOutputDirectoryError(staticPrefixPath: Path): Throwable = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index 34b8cba6e90a8..743b883375ffa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -88,7 +88,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq('arr.array(StringType)) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == s""" - |Cannot up cast array element from "STRING" to "BIGINT". + |[CANNOT_UP_CAST_DATATYPE] Cannot up cast array element from "STRING" to "BIGINT". |The type path of the target object is: |- array element class: "scala.Long" |- field (class: "scala.Array", name: "arr") @@ -211,7 +211,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq(attr) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == s""" - |Cannot up cast a from "${attr.dataType.sql}" to "STRING". + |[CANNOT_UP_CAST_DATATYPE] Cannot up cast a from "${attr.dataType.sql}" to "STRING". |The type path of the target object is: |- root class: "java.lang.String" |You can either add an explicit cast to the input data or choose a higher precision type @@ -225,7 +225,7 @@ class EncoderResolutionSuite extends PlanTest { }.message assert(msg1 == s""" - |Cannot up cast b from "BIGINT" to "INT". + |[CANNOT_UP_CAST_DATATYPE] Cannot up cast b from "BIGINT" to "INT". |The type path of the target object is: |- field (class: "scala.Int", name: "b") |- root class: "org.apache.spark.sql.catalyst.encoders.StringIntClass" @@ -238,7 +238,7 @@ class EncoderResolutionSuite extends PlanTest { }.message assert(msg2 == s""" - |Cannot up cast b.`b` from "DECIMAL(38,18)" to "BIGINT". + |[CANNOT_UP_CAST_DATATYPE] Cannot up cast b.`b` from "DECIMAL(38,18)" to "BIGINT". |The type path of the target object is: |- field (class: "scala.Long", name: "b") |- field (class: "org.apache.spark.sql.catalyst.encoders.StringLongClass", name: "b") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala index 7f4bb12ebb19b..687c67d584dd7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala @@ -188,7 +188,7 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { Optimize.execute(query.analyze) } assert(e.message == - "The feature is not supported: " + + "[UNSUPPORTED_FEATURE] The feature is not supported: " + s"""Using PythonUDF in join condition of join type ${joinType.sql} is not supported.""") val query2 = testRelationLeft.join( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 74be483cd7c37..d773d4bd271b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -133,7 +133,7 @@ case class InsertIntoHadoopFsRelationCommand( case (SaveMode.Ignore, exists) => !exists case (s, exists) => - throw QueryExecutionErrors.unsupportedSaveModeError(s.toString, exists) + throw QueryExecutionErrors.saveModeUnsupportedError(s, exists) } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out index cdc225b191366..889523537f448 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out @@ -168,7 +168,7 @@ select element_at(array(1, 2, 3), 5) struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX_IN_ELEMENT_AT] The index 5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -177,7 +177,7 @@ select element_at(array(1, 2, 3), -5) struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index -5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX_IN_ELEMENT_AT] The index -5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -195,7 +195,7 @@ select elt(4, '123', '456') struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 4 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index 4 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -204,7 +204,7 @@ select elt(0, '123', '456') struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 0 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index 0 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -213,7 +213,7 @@ select elt(-1, '123', '456') struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index -1 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index -1 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -254,7 +254,7 @@ select array(1, 2, 3)[5] struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 5 is out of bounds. The array has 3 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index 5 is out of bounds. The array has 3 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -263,7 +263,7 @@ select array(1, 2, 3)[-1] struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index -1 is out of bounds. The array has 3 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index -1 is out of bounds. The array has 3 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -337,7 +337,7 @@ select element_at(array(1, 2, 3), 5) struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX_IN_ELEMENT_AT] The index 5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -346,7 +346,7 @@ select element_at(array(1, 2, 3), -5) struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index -5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX_IN_ELEMENT_AT] The index -5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -364,7 +364,7 @@ select elt(4, '123', '456') struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 4 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index 4 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -373,7 +373,7 @@ select elt(0, '123', '456') struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index 0 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index 0 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -382,4 +382,4 @@ select elt(-1, '123', '456') struct<> -- !query output org.apache.spark.SparkArrayIndexOutOfBoundsException -The index -1 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_ARRAY_INDEX] The index -1 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out index 1bba7e0bb4ee9..f1c2dbd3d7ded 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out @@ -8,7 +8,7 @@ SELECT CAST('1.23' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1.23' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1.23' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('1.23' AS int) ^^^^^^^^^^^^^^^^^^^ @@ -20,7 +20,7 @@ SELECT CAST('1.23' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1.23' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1.23' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('1.23' AS long) ^^^^^^^^^^^^^^^^^^^^ @@ -32,7 +32,7 @@ SELECT CAST('-4.56' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '-4.56' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '-4.56' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('-4.56' AS int) ^^^^^^^^^^^^^^^^^^^^ @@ -44,7 +44,7 @@ SELECT CAST('-4.56' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '-4.56' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '-4.56' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('-4.56' AS long) ^^^^^^^^^^^^^^^^^^^^^ @@ -56,7 +56,7 @@ SELECT CAST('abc' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'abc' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'abc' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('abc' AS int) ^^^^^^^^^^^^^^^^^^ @@ -68,7 +68,7 @@ SELECT CAST('abc' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'abc' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'abc' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('abc' AS long) ^^^^^^^^^^^^^^^^^^^ @@ -80,7 +80,7 @@ SELECT CAST('abc' AS float) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'abc' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'abc' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('abc' AS float) ^^^^^^^^^^^^^^^^^^^^ @@ -92,7 +92,7 @@ SELECT CAST('abc' AS double) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'abc' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'abc' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('abc' AS double) ^^^^^^^^^^^^^^^^^^^^^ @@ -104,7 +104,7 @@ SELECT CAST('1234567890123' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1234567890123' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1234567890123' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('1234567890123' AS int) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -116,7 +116,7 @@ SELECT CAST('12345678901234567890123' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '12345678901234567890123' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '12345678901234567890123' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('12345678901234567890123' AS long) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -128,7 +128,7 @@ SELECT CAST('' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('' AS int) ^^^^^^^^^^^^^^^ @@ -140,7 +140,7 @@ SELECT CAST('' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('' AS long) ^^^^^^^^^^^^^^^^ @@ -152,7 +152,7 @@ SELECT CAST('' AS float) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('' AS float) ^^^^^^^^^^^^^^^^^ @@ -164,7 +164,7 @@ SELECT CAST('' AS double) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('' AS double) ^^^^^^^^^^^^^^^^^^ @@ -192,7 +192,7 @@ SELECT CAST('123.a' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '123.a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '123.a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('123.a' AS int) ^^^^^^^^^^^^^^^^^^^^ @@ -204,7 +204,7 @@ SELECT CAST('123.a' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '123.a' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '123.a' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('123.a' AS long) ^^^^^^^^^^^^^^^^^^^^^ @@ -216,7 +216,7 @@ SELECT CAST('123.a' AS float) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '123.a' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '123.a' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('123.a' AS float) ^^^^^^^^^^^^^^^^^^^^^^ @@ -228,7 +228,7 @@ SELECT CAST('123.a' AS double) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '123.a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '123.a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('123.a' AS double) ^^^^^^^^^^^^^^^^^^^^^^^ @@ -248,7 +248,7 @@ SELECT CAST('-2147483649' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '-2147483649' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '-2147483649' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('-2147483649' AS int) ^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -268,7 +268,7 @@ SELECT CAST('2147483648' AS int) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '2147483648' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '2147483648' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('2147483648' AS int) ^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -288,7 +288,7 @@ SELECT CAST('-9223372036854775809' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '-9223372036854775809' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '-9223372036854775809' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('-9223372036854775809' AS long) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -308,7 +308,7 @@ SELECT CAST('9223372036854775808' AS long) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '9223372036854775808' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '9223372036854775808' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT CAST('9223372036854775808' AS long) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -567,7 +567,7 @@ select cast('1中文' as tinyint) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1中文' of the type "STRING" cannot be cast to "TINYINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1中文' of the type "STRING" cannot be cast to "TINYINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('1中文' as tinyint) ^^^^^^^^^^^^^^^^^^^^^^ @@ -579,7 +579,7 @@ select cast('1中文' as smallint) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1中文' of the type "STRING" cannot be cast to "SMALLINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1中文' of the type "STRING" cannot be cast to "SMALLINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('1中文' as smallint) ^^^^^^^^^^^^^^^^^^^^^^^ @@ -591,7 +591,7 @@ select cast('1中文' as INT) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1中文' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1中文' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('1中文' as INT) ^^^^^^^^^^^^^^^^^^ @@ -603,7 +603,7 @@ select cast('中文1' as bigint) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '中文1' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '中文1' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('中文1' as bigint) ^^^^^^^^^^^^^^^^^^^^^ @@ -615,7 +615,7 @@ select cast('1中文' as bigint) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1中文' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1中文' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('1中文' as bigint) ^^^^^^^^^^^^^^^^^^^^^ @@ -645,7 +645,7 @@ select cast('\t\n xyz \t\r' as boolean) struct<> -- !query output org.apache.spark.SparkRuntimeException -The value ' +[CAST_INVALID_INPUT] The value ' xyz ' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('\t\n xyz \t\r' as boolean) @@ -666,7 +666,7 @@ select cast('123.45' as decimal(4, 2)) struct<> -- !query output org.apache.spark.SparkArithmeticException -Decimal(expanded, 123.45, 5, 2) cannot be represented as Decimal(4, 2). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CANNOT_CHANGE_DECIMAL_PRECISION] Decimal(expanded, 123.45, 5, 2) cannot be represented as Decimal(4, 2). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('123.45' as decimal(4, 2)) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -678,7 +678,7 @@ select cast('xyz' as decimal(4, 2)) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'xyz' of the type "STRING" cannot be cast to "DECIMAL(4,2)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'xyz' of the type "STRING" cannot be cast to "DECIMAL(4,2)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('xyz' as decimal(4, 2)) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -698,7 +698,7 @@ select cast('a' as date) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value 'a' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('a' as date) ^^^^^^^^^^^^^^^^^ @@ -718,7 +718,7 @@ select cast('a' as timestamp) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value 'a' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('a' as timestamp) ^^^^^^^^^^^^^^^^^^^^^^ @@ -738,7 +738,7 @@ select cast('a' as timestamp_ntz) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value 'a' of the type "STRING" cannot be cast to "TIMESTAMP_NTZ" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "TIMESTAMP_NTZ" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast('a' as timestamp_ntz) ^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -750,7 +750,7 @@ select cast(cast('inf' as double) as timestamp) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value Infinity of the type "DOUBLE" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value Infinity of the type "DOUBLE" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast(cast('inf' as double) as timestamp) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -762,7 +762,7 @@ select cast(cast('inf' as float) as timestamp) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value Infinity of the type "DOUBLE" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value Infinity of the type "DOUBLE" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast(cast('inf' as float) as timestamp) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ 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 0b3f408164fba..ab77d309b2387 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 @@ -232,7 +232,7 @@ select next_day("xx", "Mon") struct<> -- !query output org.apache.spark.SparkDateTimeException -The value 'xx' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'xx' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select next_day("xx", "Mon") ^^^^^^^^^^^^^^^^^^^^^ @@ -327,7 +327,7 @@ select date_add('2011-11-11', '1.2') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select date_add('2011-11-11', '1.2') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -438,7 +438,7 @@ select date_sub(date'2011-11-11', '1.2') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value '1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select date_sub(date'2011-11-11', '1.2') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -650,7 +650,7 @@ select to_date('26/October/2015', 'dd/MMMMM/yyyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -659,7 +659,7 @@ select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMM struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -668,7 +668,7 @@ select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !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 d69477dd327e8..a646bf8fdf234 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 @@ -17,7 +17,7 @@ select to_timestamp('1', 'yy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -35,7 +35,7 @@ select to_timestamp('123', 'yy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -44,7 +44,7 @@ select to_timestamp('1', 'yyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -53,7 +53,7 @@ select to_timestamp('1234567', 'yyyyyyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -71,7 +71,7 @@ select to_timestamp('9', 'DD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -80,7 +80,7 @@ select to_timestamp('9', 'DDD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -89,7 +89,7 @@ select to_timestamp('99', 'DDD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -161,7 +161,7 @@ select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -242,7 +242,7 @@ select cast("Unparseable" as timestamp) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value 'Unparseable' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'Unparseable' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast("Unparseable" as timestamp) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -254,7 +254,7 @@ select cast("Unparseable" as date) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value 'Unparseable' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'Unparseable' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select cast("Unparseable" as date) ^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index 9025e91064888..62dc32b4060d4 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -76,7 +76,7 @@ select (5e36BD + 0.1) + 5e36BD struct<> -- !query output org.apache.spark.SparkArithmeticException -Decimal(expanded, 10000000000000000000000000000000000000.1, 39, 1) cannot be represented as Decimal(38, 1). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CANNOT_CHANGE_DECIMAL_PRECISION] Decimal(expanded, 10000000000000000000000000000000000000.1, 39, 1) cannot be represented as Decimal(38, 1). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select (5e36BD + 0.1) + 5e36BD ^^^^^^^^^^^^^^^^^^^^^^^ @@ -88,7 +88,7 @@ select (-4e36BD - 0.1) - 7e36BD struct<> -- !query output org.apache.spark.SparkArithmeticException -Decimal(expanded, -11000000000000000000000000000000000000.1, 39, 1) cannot be represented as Decimal(38, 1). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CANNOT_CHANGE_DECIMAL_PRECISION] Decimal(expanded, -11000000000000000000000000000000000000.1, 39, 1) cannot be represented as Decimal(38, 1). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select (-4e36BD - 0.1) - 7e36BD ^^^^^^^^^^^^^^^^^^^^^^^^ @@ -100,7 +100,7 @@ select 12345678901234567890.0 * 12345678901234567890.0 struct<> -- !query output org.apache.spark.SparkArithmeticException -Decimal(expanded, 152415787532388367501905199875019052100, 39, 0) cannot be represented as Decimal(38, 2). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CANNOT_CHANGE_DECIMAL_PRECISION] Decimal(expanded, 152415787532388367501905199875019052100, 39, 0) cannot be represented as Decimal(38, 2). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select 12345678901234567890.0 * 12345678901234567890.0 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -112,7 +112,7 @@ select 1e35BD / 0.1 struct<> -- !query output org.apache.spark.SparkArithmeticException -Decimal(expanded, 1000000000000000000000000000000000000, 37, 0) cannot be represented as Decimal(38, 6). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CANNOT_CHANGE_DECIMAL_PRECISION] Decimal(expanded, 1000000000000000000000000000000000000, 37, 0) cannot be represented as Decimal(38, 6). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select 1e35BD / 0.1 ^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index f4ec0afb0cc2c..d0f2be0d51974 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -122,7 +122,7 @@ select interval 2 second * 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select interval 2 second * 'a' ^^^^^^^^^^^^^^^^^^^^^^^ @@ -134,7 +134,7 @@ select interval 2 second / 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select interval 2 second / 'a' ^^^^^^^^^^^^^^^^^^^^^^^ @@ -146,7 +146,7 @@ select interval 2 year * 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select interval 2 year * 'a' ^^^^^^^^^^^^^^^^^^^^^ @@ -158,7 +158,7 @@ select interval 2 year / 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select interval 2 year / 'a' ^^^^^^^^^^^^^^^^^^^^^ @@ -186,7 +186,7 @@ select 'a' * interval 2 second struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select 'a' * interval 2 second ^^^^^^^^^^^^^^^^^^^^^^^ @@ -198,7 +198,7 @@ select 'a' * interval 2 year struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select 'a' * interval 2 year ^^^^^^^^^^^^^^^^^^^^^ @@ -228,7 +228,7 @@ select interval '2 seconds' / 0 struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select interval '2 seconds' / 0 ^^^^^^^^^^^^^^^^^^^^^^^^ @@ -264,7 +264,7 @@ select interval '2' year / 0 struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select interval '2' year / 0 ^^^^^^^^^^^^^^^^^^^^^ @@ -664,7 +664,7 @@ select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) struct<> -- !query output org.apache.spark.SparkArithmeticException -Decimal(expanded, 1234567890123456789, 20, 0) cannot be represented as Decimal(18, 6). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CANNOT_CHANGE_DECIMAL_PRECISION] Decimal(expanded, 1234567890123456789, 20, 0) cannot be represented as Decimal(18, 6). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -1370,7 +1370,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'day': extra input 'day'(line 1, pos 27) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'day': extra input 'day'(line 1, pos 27) == SQL == select interval 30 day day day @@ -1402,7 +1402,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'days': extra input 'days'(line 1, pos 29) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'days': extra input 'days'(line 1, pos 29) == SQL == select interval 30 days days days @@ -1516,7 +1516,7 @@ select '4 11:11' - interval '4 22:12' day to minute struct<> -- !query output org.apache.spark.SparkDateTimeException -The value '4 11:11' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '4 11:11' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select '4 11:11' - interval '4 22:12' day to minute ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -1528,7 +1528,7 @@ select '4 12:12:12' + interval '4 22:12' day to minute struct<> -- !query output org.apache.spark.SparkDateTimeException -The value '4 12:12:12' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '4 12:12:12' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select '4 12:12:12' + interval '4 22:12' day to minute ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -1566,7 +1566,7 @@ select str - interval '4 22:12' day to minute from interval_view struct<> -- !query output org.apache.spark.SparkDateTimeException -The value '1' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select str - interval '4 22:12' day to minute from interval_view ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -1578,7 +1578,7 @@ select str + interval '4 22:12' day to minute from interval_view struct<> -- !query output org.apache.spark.SparkDateTimeException -The value '1' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select str + interval '4 22:12' day to minute from interval_view ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -1789,7 +1789,7 @@ select -(a) from values (interval '-2147483648 months', interval '2147483647 mon struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -1798,7 +1798,7 @@ select a - b from values (interval '-2147483648 months', interval '2147483647 mo struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -1807,7 +1807,7 @@ select b + interval '1 month' from values (interval '-2147483648 months', interv struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -2036,7 +2036,7 @@ SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -2048,7 +2048,7 @@ SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -2094,7 +2094,7 @@ SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -2106,7 +2106,7 @@ SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index f13542dd4424c..f6341caed81c6 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -217,7 +217,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near '.'(line 1, pos 7) +[PARSE_SYNTAX_ERROR] Syntax error at or near '.'(line 1, pos 7) == SQL == select .e3 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out index 9e37402dd470f..14df691c0039f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out @@ -8,7 +8,7 @@ select element_at(map(1, 'a', 2, 'b'), 5) struct<> -- !query output org.apache.spark.SparkNoSuchElementException -Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[MAP_KEY_DOES_NOT_EXIST] Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select element_at(map(1, 'a', 2, 'b'), 5) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -20,7 +20,7 @@ select map(1, 'a', 2, 'b')[5] struct<> -- !query output org.apache.spark.SparkNoSuchElementException -Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[MAP_KEY_DOES_NOT_EXIST] Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select map(1, 'a', 2, 'b')[5] ^^^^^^^^^^^^^^^^^^^^^^ @@ -114,7 +114,7 @@ select element_at(map(1, 'a', 2, 'b'), 5) struct<> -- !query output org.apache.spark.SparkNoSuchElementException -Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[MAP_KEY_DOES_NOT_EXIST] Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select element_at(map(1, 'a', 2, 'b'), 5) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -126,7 +126,7 @@ select element_at(map('a', 1, 'b', 2), 'c') struct<> -- !query output org.apache.spark.SparkNoSuchElementException -Key 'c' does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[MAP_KEY_DOES_NOT_EXIST] Key 'c' does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select element_at(map('a', 1, 'b', 2), 'c') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index 35ec3a9756602..5193eaf5f4bde 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -82,7 +82,7 @@ select left("abcd", -2), left("abcd", 0), left("abcd", 'a') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 43) == ...t("abcd", -2), left("abcd", 0), left("abcd", 'a') ^^^^^^^^^^^^^^^^^ @@ -110,7 +110,7 @@ select right("abcd", -2), right("abcd", 0), right("abcd", 'a') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 45) == ...("abcd", -2), right("abcd", 0), right("abcd", 'a') ^^^^^^^^^^^^^^^^^^ @@ -419,7 +419,7 @@ SELECT lpad('hi', 'invalid_length') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'invalid_length' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'invalid_length' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT lpad('hi', 'invalid_length') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -431,7 +431,7 @@ SELECT rpad('hi', 'invalid_length') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'invalid_length' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'invalid_length' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT rpad('hi', 'invalid_length') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ 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 368cab2eaeac3..9ba45bb7ebac3 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 @@ -98,7 +98,7 @@ SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) struct<> -- !query output org.apache.spark.SparkDateTimeException -The fraction of sec must be zero. Valid range is [0, 60]. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_FRACTION_OF_SECOND] The fraction of sec must be zero. Valid range is [0, 60]. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -733,7 +733,7 @@ select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -742,7 +742,7 @@ select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -751,7 +751,7 @@ select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -760,7 +760,7 @@ select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -769,7 +769,7 @@ select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat' struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -778,7 +778,7 @@ select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMM struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out index 6c1edfed38b76..03ba62c314a35 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out @@ -161,7 +161,7 @@ SELECT db1.t1.i1 FROM t1, mydb2.t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'db1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1]; line 1 pos 7 +[MISSING_COLUMN] Column 'db1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1]; line 1 pos 7 -- !query @@ -186,7 +186,7 @@ SELECT mydb1.t1 FROM t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'mydb1.t1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7 +[MISSING_COLUMN] Column 'mydb1.t1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7 -- !query @@ -204,7 +204,7 @@ SELECT t1 FROM mydb1.t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7 +[MISSING_COLUMN] Column 't1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7 -- !query @@ -221,7 +221,7 @@ SELECT mydb1.t1.i1 FROM t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'mydb1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1]; line 1 pos 7 +[MISSING_COLUMN] Column 'mydb1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1]; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index 92b454bd2f6aa..d6a3de30f63ef 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -34,7 +34,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException Cannot parse the data type: -Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2) == SQL == a InvalidType diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index 6b572460b957f..fd6eb833d664d 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -133,7 +133,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near ')'(line 1, pos 7) +[PARSE_SYNTAX_ERROR] Syntax error at or near ')'(line 1, pos 7) == SQL == WITH t() AS (SELECT 1) diff --git a/sql/core/src/test/resources/sql-tests/results/date.sql.out b/sql/core/src/test/resources/sql-tests/results/date.sql.out index 83f59cbca2ae7..49ca02502798f 100644 --- a/sql/core/src/test/resources/sql-tests/results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date.sql.out @@ -319,7 +319,7 @@ select date_add('2011-11-11', '1.2') struct<> -- !query output org.apache.spark.sql.AnalysisException -The second argument of 'date_add' function needs to be an integer. +[SECOND_FUNCTION_ARGUMENT_NOT_INTEGER] The second argument of 'date_add' function needs to be an integer. -- !query @@ -427,7 +427,7 @@ select date_sub(date'2011-11-11', '1.2') struct<> -- !query output org.apache.spark.sql.AnalysisException -The second argument of 'date_sub' function needs to be an integer. +[SECOND_FUNCTION_ARGUMENT_NOT_INTEGER] The second argument of 'date_sub' function needs to be an integer. -- !query @@ -640,7 +640,7 @@ select to_date('26/October/2015', 'dd/MMMMM/yyyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -649,7 +649,7 @@ select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMM struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -658,7 +658,7 @@ select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out index 6649ae3dbaf1c..d04afc7e9c481 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out @@ -8,7 +8,7 @@ select date_format('2018-11-17 13:33:33.333', 'GGGGG') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -17,7 +17,7 @@ select date_format('2018-11-17 13:33:33.333', 'yyyyyyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -44,7 +44,7 @@ select date_format('2018-11-17 13:33:33.333', 'MMMMM') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -53,7 +53,7 @@ select date_format('2018-11-17 13:33:33.333', 'LLLLL') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'LLLLL' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'LLLLL' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -62,7 +62,7 @@ select date_format('2018-11-17 13:33:33.333', 'EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -71,7 +71,7 @@ select date_format('2018-11-17 13:33:33.333', 'FF') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'FF' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'FF' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -80,7 +80,7 @@ select date_format('2018-11-17 13:33:33.333', 'ddd') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'ddd' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'ddd' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -89,7 +89,7 @@ select date_format('2018-11-17 13:33:33.333', 'DDDD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'DDDD' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'DDDD' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -98,7 +98,7 @@ select date_format('2018-11-17 13:33:33.333', 'HHH') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'HHH' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'HHH' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -107,7 +107,7 @@ select date_format('2018-11-17 13:33:33.333', 'hhh') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'hhh' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'hhh' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -116,7 +116,7 @@ select date_format('2018-11-17 13:33:33.333', 'kkk') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'kkk' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'kkk' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -125,7 +125,7 @@ select date_format('2018-11-17 13:33:33.333', 'KKK') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'KKK' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'KKK' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -134,7 +134,7 @@ select date_format('2018-11-17 13:33:33.333', 'mmm') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'mmm' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'mmm' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -143,7 +143,7 @@ select date_format('2018-11-17 13:33:33.333', 'sss') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'sss' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'sss' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -152,7 +152,7 @@ select date_format('2018-11-17 13:33:33.333', 'SSSSSSSSSS') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'SSSSSSSSSS' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'SSSSSSSSSS' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -161,7 +161,7 @@ select date_format('2018-11-17 13:33:33.333', 'aa') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -179,7 +179,7 @@ select date_format('2018-11-17 13:33:33.333', 'zzzzz') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'zzzzz' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'zzzzz' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -197,7 +197,7 @@ select date_format('2018-11-17 13:33:33.333', 'ZZZZZZ') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'ZZZZZZ' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'ZZZZZZ' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -260,7 +260,7 @@ select date_format('2018-11-17 13:33:33.333', 'Y') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'Y' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'Y' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -269,7 +269,7 @@ select date_format('2018-11-17 13:33:33.333', 'w') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'w' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'w' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -278,7 +278,7 @@ select date_format('2018-11-17 13:33:33.333', 'W') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'W' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'W' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -287,7 +287,7 @@ select date_format('2018-11-17 13:33:33.333', 'u') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'u' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'u' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query 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 03ec7957ed668..54351adf20059 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 @@ -319,7 +319,7 @@ select date_add('2011-11-11', '1.2') struct<> -- !query output org.apache.spark.sql.AnalysisException -The second argument of 'date_add' function needs to be an integer. +[SECOND_FUNCTION_ARGUMENT_NOT_INTEGER] The second argument of 'date_add' function needs to be an integer. -- !query @@ -427,7 +427,7 @@ select date_sub(date'2011-11-11', '1.2') struct<> -- !query output org.apache.spark.sql.AnalysisException -The second argument of 'date_sub' function needs to be an integer. +[SECOND_FUNCTION_ARGUMENT_NOT_INTEGER] The second argument of 'date_sub' function needs to be an integer. -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index 9fc28876a5b2a..e453105b10e30 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -17,7 +17,7 @@ select to_timestamp('1', 'yy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -34,7 +34,7 @@ select to_timestamp('123', 'yy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -43,7 +43,7 @@ select to_timestamp('1', 'yyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -52,7 +52,7 @@ select to_timestamp('1234567', 'yyyyyyy') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -69,7 +69,7 @@ select to_timestamp('9', 'DD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -78,7 +78,7 @@ select to_timestamp('9', 'DDD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -87,7 +87,7 @@ select to_timestamp('99', 'DDD') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -152,7 +152,7 @@ select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out index 322b24877a57e..041116974cbf5 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out @@ -112,7 +112,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'desc_temp1'(line 1, pos 21) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'desc_temp1'(line 1, pos 21) == SQL == DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') @@ -126,7 +126,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'desc_temp1'(line 1, pos 21) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'desc_temp1'(line 1, pos 21) == SQL == DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 @@ -143,7 +143,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'insert'(line 3, pos 5) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'insert'(line 3, pos 5) == SQL == DESCRIBE diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out index f249908163d01..98d93868457fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -499,7 +499,7 @@ SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(cours struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query @@ -508,7 +508,7 @@ SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(co struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query @@ -563,7 +563,7 @@ SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(cou struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query @@ -572,7 +572,7 @@ SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID( struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index ba06b148d2dee..c7315a7354f18 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -187,7 +187,7 @@ SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'non_existing' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 21 +[MISSING_COLUMN] Column 'non_existing' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 21 -- !query @@ -231,7 +231,7 @@ SELECT a AS k, COUNT(b) FROM testData GROUP BY k struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 47 +[MISSING_COLUMN] Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 47 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out index 21c13af560dac..c0f1a6975d1a3 100644 --- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -138,7 +138,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'ROLLUP': extra input 'ROLLUP'(line 1, pos 53) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'ROLLUP': extra input 'ROLLUP'(line 1, pos 53) == SQL == SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP @@ -152,7 +152,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'CUBE': extra input 'CUBE'(line 1, pos 53) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'CUBE': extra input 'CUBE'(line 1, pos 53) == SQL == SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 71fb0c0845d39..1cbe117a05b16 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -204,7 +204,7 @@ select interval '2 seconds' / 0 struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select interval '2 seconds' / 0 ^^^^^^^^^^^^^^^^^^^^^^^^ @@ -240,7 +240,7 @@ select interval '2' year / 0 struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select interval '2' year / 0 ^^^^^^^^^^^^^^^^^^^^^ @@ -1342,7 +1342,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'day': extra input 'day'(line 1, pos 27) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'day': extra input 'day'(line 1, pos 27) == SQL == select interval 30 day day day @@ -1374,7 +1374,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'days': extra input 'days'(line 1, pos 29) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'days': extra input 'days'(line 1, pos 29) == SQL == select interval 30 days days days @@ -1745,7 +1745,7 @@ select -(a) from values (interval '-2147483648 months', interval '2147483647 mon struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -1754,7 +1754,7 @@ select a - b from values (interval '-2147483648 months', interval '2147483647 mo struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -1763,7 +1763,7 @@ select b + interval '1 month' from values (interval '-2147483648 months', interv struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -1992,7 +1992,7 @@ SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -2004,7 +2004,7 @@ SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -2050,7 +2050,7 @@ SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -2062,7 +2062,7 @@ SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L struct<> -- !query output org.apache.spark.SparkArithmeticException -Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index cc1619813dd55..5b36a7959aefb 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out @@ -153,7 +153,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14) +[UNSUPPORTED_FEATURE] The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14) == SQL == SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) @@ -167,7 +167,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -The feature is not supported: LATERAL join with USING join.(line 1, pos 14) +[UNSUPPORTED_FEATURE] The feature is not supported: LATERAL join with USING join.(line 1, pos 14) == SQL == SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) @@ -269,7 +269,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't2.c1' does not exist. Did you mean one of the following? []; line 1 pos 50 +[MISSING_COLUMN] Column 't2.c1' does not exist. Did you mean one of the following? []; line 1 pos 50 -- !query @@ -387,7 +387,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't1.c1' does not exist. Did you mean one of the following? []; line 1 pos 61 +[MISSING_COLUMN] Column 't1.c1' does not exist. Did you mean one of the following? []; line 1 pos 61 -- !query @@ -396,7 +396,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'c2' does not exist. Did you mean one of the following? []; line 1 pos 70 +[MISSING_COLUMN] Column 'c2' does not exist. Did you mean one of the following? []; line 1 pos 70 -- !query @@ -423,7 +423,7 @@ SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't1.c1' does not exist. Did you mean one of the following? [spark_catalog.default.t2.c1, spark_catalog.default.t2.c2]; line 1 pos 73 +[MISSING_COLUMN] Column 't1.c1' does not exist. Did you mean one of the following? [spark_catalog.default.t2.c1, spark_catalog.default.t2.c2]; line 1 pos 73 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 48a7b7b7952ce..6f1a5c3bd5bb7 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -125,7 +125,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException Cannot parse the data type: -Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2) == SQL == a InvalidType @@ -339,7 +339,7 @@ select from_json( struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '02-29' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '02-29' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query @@ -351,7 +351,7 @@ select from_json( struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '02-29' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '02-29' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index f13542dd4424c..f6341caed81c6 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -217,7 +217,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near '.'(line 1, pos 7) +[PARSE_SYNTAX_ERROR] Syntax error at or near '.'(line 1, pos 7) == SQL == select .e3 diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out index 3686776c71704..a146ee74b8ed1 100644 --- a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -232,7 +232,7 @@ SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'nt2.k' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.k, __auto_generated_subquery_name.v1, __auto_generated_subquery_name.v2]; line 1 pos 7 +[MISSING_COLUMN] Column 'nt2.k' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.k, __auto_generated_subquery_name.v1, __auto_generated_subquery_name.v2]; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index 0a42750d24571..ef20f8fe7e605 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -232,7 +232,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 +[MISSING_COLUMN] Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 -- !query @@ -313,7 +313,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct +[PIVOT_VALUE_DATA_TYPE_MISMATCH] Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct -- !query @@ -326,7 +326,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15 +[MISSING_COLUMN] Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15 -- !query @@ -339,7 +339,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Literal expressions required for pivot values, found "course". +[NON_LITERAL_PIVOT_VALUES] Literal expressions required for pivot values, found "course". -- !query @@ -458,7 +458,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid pivot column `__auto_generated_subquery_name`.`m`. Pivot columns must be comparable. +[INCOMPARABLE_PIVOT_COLUMN] Invalid pivot column `__auto_generated_subquery_name`.`m`. Pivot columns must be comparable. -- !query @@ -475,7 +475,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid pivot column `named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`. Pivot columns must be comparable. +[INCOMPARABLE_PIVOT_COLUMN] Invalid pivot column `named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`. Pivot columns must be comparable. -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index 5f4ea7aef702e..91b2a616263e7 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -414,4 +414,4 @@ from tenk1 o struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 63 +[MISSING_COLUMN] Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 63 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index f68d92baee209..af8cc85255478 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -56,7 +56,7 @@ SELECT boolean('test') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'test' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'test' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('test') AS error ^^^^^^^^^^^^^^^ @@ -76,7 +76,7 @@ SELECT boolean('foo') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'foo' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'foo' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('foo') AS error ^^^^^^^^^^^^^^ @@ -104,7 +104,7 @@ SELECT boolean('yeah') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'yeah' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'yeah' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('yeah') AS error ^^^^^^^^^^^^^^^ @@ -132,7 +132,7 @@ SELECT boolean('nay') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'nay' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'nay' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('nay') AS error ^^^^^^^^^^^^^^ @@ -144,7 +144,7 @@ SELECT boolean('on') AS true struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'on' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'on' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('on') AS true ^^^^^^^^^^^^^ @@ -156,7 +156,7 @@ SELECT boolean('off') AS `false` struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'off' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'off' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('off') AS `false` ^^^^^^^^^^^^^^ @@ -168,7 +168,7 @@ SELECT boolean('of') AS `false` struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'of' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'of' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('of') AS `false` ^^^^^^^^^^^^^ @@ -180,7 +180,7 @@ SELECT boolean('o') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'o' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'o' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('o') AS error ^^^^^^^^^^^^ @@ -192,7 +192,7 @@ SELECT boolean('on_') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'on_' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'on_' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('on_') AS error ^^^^^^^^^^^^^^ @@ -204,7 +204,7 @@ SELECT boolean('off_') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value 'off_' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'off_' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('off_') AS error ^^^^^^^^^^^^^^^ @@ -224,7 +224,7 @@ SELECT boolean('11') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value '11' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '11' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('11') AS error ^^^^^^^^^^^^^ @@ -244,7 +244,7 @@ SELECT boolean('000') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value '000' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '000' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('000') AS error ^^^^^^^^^^^^^^ @@ -256,7 +256,7 @@ SELECT boolean('') AS error struct<> -- !query output org.apache.spark.SparkRuntimeException -The value '' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean('') AS error ^^^^^^^^^^^ @@ -365,7 +365,7 @@ SELECT boolean(string(' tru e ')) AS invalid struct<> -- !query output org.apache.spark.SparkRuntimeException -The value ' tru e ' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value ' tru e ' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean(string(' tru e ')) AS invalid ^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -377,7 +377,7 @@ SELECT boolean(string('')) AS invalid struct<> -- !query output org.apache.spark.SparkRuntimeException -The value '' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT boolean(string('')) AS invalid ^^^^^^^^^^^^^^^^^^^ @@ -524,7 +524,7 @@ INSERT INTO BOOLTBL2 struct<> -- !query output org.apache.spark.sql.AnalysisException -failed to evaluate expression CAST('XXX' AS BOOLEAN): The value 'XXX' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +failed to evaluate expression CAST('XXX' AS BOOLEAN): [CAST_INVALID_INPUT] The value 'XXX' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 2, position 12) == VALUES (boolean('XXX')) ^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index a2064a4e20cec..a3637eca9588d 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -66,7 +66,7 @@ CREATE VIEW key_dependent_view_no_cols AS struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'FROM' does not exist. Did you mean one of the following? []; line 2 pos 10 +[MISSING_COLUMN] Column 'FROM' does not exist. Did you mean one of the following? []; line 2 pos 10 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index d411c7bc4699f..9f946f40e7785 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -96,7 +96,7 @@ SELECT float('N A N') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'N A N' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'N A N' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT float('N A N') ^^^^^^^^^^^^^^ @@ -108,7 +108,7 @@ SELECT float('NaN x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'NaN x' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'NaN x' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT float('NaN x') ^^^^^^^^^^^^^^ @@ -120,7 +120,7 @@ SELECT float(' INFINITY x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value ' INFINITY x' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value ' INFINITY x' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT float(' INFINITY x') ^^^^^^^^^^^^^^^^^^^^^^^ @@ -156,7 +156,7 @@ SELECT float(decimal('nan')) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'nan' of the type "STRING" cannot be cast to "DECIMAL(10,0)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'nan' of the type "STRING" cannot be cast to "DECIMAL(10,0)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 14) == SELECT float(decimal('nan')) ^^^^^^^^^^^^^^ @@ -340,7 +340,7 @@ SELECT int(float('2147483647')) struct<> -- !query output org.apache.spark.SparkArithmeticException -The value 2.14748365E9 of the type "FLOAT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value 2.14748365E9 of the type "FLOAT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -357,7 +357,7 @@ SELECT int(float('-2147483900')) struct<> -- !query output org.apache.spark.SparkArithmeticException -The value -2.1474839E9 of the type "FLOAT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value -2.1474839E9 of the type "FLOAT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -390,7 +390,7 @@ SELECT bigint(float('-9223380000000000000')) struct<> -- !query output org.apache.spark.SparkArithmeticException -The value -9.22338E18 of the type "FLOAT" cannot be cast to "BIGINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value -9.22338E18 of the type "FLOAT" cannot be cast to "BIGINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. 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/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out index b00a0d094636b..3c6f903f2cda3 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out @@ -128,7 +128,7 @@ SELECT double('N A N') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'N A N' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'N A N' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT double('N A N') ^^^^^^^^^^^^^^^ @@ -140,7 +140,7 @@ SELECT double('NaN x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'NaN x' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'NaN x' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT double('NaN x') ^^^^^^^^^^^^^^^ @@ -152,7 +152,7 @@ SELECT double(' INFINITY x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value ' INFINITY x' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value ' INFINITY x' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == SELECT double(' INFINITY x') ^^^^^^^^^^^^^^^^^^^^^^^^ @@ -188,7 +188,7 @@ SELECT double(decimal('nan')) struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'nan' of the type "STRING" cannot be cast to "DECIMAL(10,0)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'nan' of the type "STRING" cannot be cast to "DECIMAL(10,0)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 15) == SELECT double(decimal('nan')) ^^^^^^^^^^^^^^ @@ -845,7 +845,7 @@ SELECT bigint(double('-9223372036854780000')) struct<> -- !query output org.apache.spark.SparkArithmeticException -The value -9.22337203685478E18D of the type "DOUBLE" cannot be cast to "BIGINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value -9.22337203685478E18D of the type "DOUBLE" cannot be cast to "BIGINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. 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/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index d8351aa6251ac..0f2f73e8ca698 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -200,7 +200,7 @@ SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 26) == SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^^^^^^ @@ -223,7 +223,7 @@ SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 26) == SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^ @@ -246,7 +246,7 @@ SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 26) == SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^^^^^^ @@ -270,7 +270,7 @@ SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 26) == SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^ @@ -294,7 +294,7 @@ SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 26) == SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^^^^^^ @@ -318,7 +318,7 @@ SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i struct<> -- !query output org.apache.spark.SparkArithmeticException -integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 26) == SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index 1b52cd1580b57..af21e8dbbe1b1 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -392,7 +392,7 @@ SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL struct<> -- !query output org.apache.spark.SparkArithmeticException -long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 29) == SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL ^^^^^^^ @@ -575,7 +575,7 @@ select bigint('9223372036854775800') / bigint('0') struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select bigint('9223372036854775800') / bigint('0') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -587,7 +587,7 @@ select bigint('-9223372036854775808') / smallint('0') struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select bigint('-9223372036854775808') / smallint('0') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -599,7 +599,7 @@ select smallint('100') / bigint('0') struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == select smallint('100') / bigint('0') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -619,7 +619,7 @@ SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 struct<> -- !query output org.apache.spark.SparkArithmeticException -The value 4567890123456789L of the type "BIGINT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value 4567890123456789L of the type "BIGINT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -636,7 +636,7 @@ SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 struct<> -- !query output org.apache.spark.SparkArithmeticException -The value 4567890123456789L of the type "BIGINT" cannot be cast to "SMALLINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value 4567890123456789L of the type "BIGINT" cannot be cast to "SMALLINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -673,7 +673,7 @@ SELECT CAST(double('922337203685477580700.0') AS bigint) struct<> -- !query output org.apache.spark.SparkArithmeticException -The value 9.223372036854776E20D of the type "DOUBLE" cannot be cast to "BIGINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value 9.223372036854776E20D of the type "DOUBLE" cannot be cast to "BIGINT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -745,7 +745,7 @@ SELECT string(int(shiftleft(bigint(-1), 63))+1) struct<> -- !query output org.apache.spark.SparkArithmeticException -The value -9223372036854775808L of the type "BIGINT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_OVERFLOW] The value -9223372036854775808L of the type "BIGINT" cannot be cast to "INT" due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -754,7 +754,7 @@ SELECT bigint((-9223372036854775808)) * bigint((-1)) struct<> -- !query output org.apache.spark.SparkArithmeticException -long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT bigint((-9223372036854775808)) * bigint((-1)) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -782,7 +782,7 @@ SELECT bigint((-9223372036854775808)) * int((-1)) struct<> -- !query output org.apache.spark.SparkArithmeticException -long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT bigint((-9223372036854775808)) * int((-1)) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -810,7 +810,7 @@ SELECT bigint((-9223372036854775808)) * smallint((-1)) struct<> -- !query output org.apache.spark.SparkArithmeticException -long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 8) == SELECT bigint((-9223372036854775808)) * smallint((-1)) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out index 6d27785e16f13..3e0ca060e8f4b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out @@ -3248,7 +3248,7 @@ select * from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 +[MISSING_COLUMN] Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 -- !query @@ -3267,7 +3267,7 @@ select t1.uunique1 from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7 +[MISSING_COLUMN] Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7 -- !query @@ -3277,7 +3277,7 @@ select t2.uunique1 from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 7 +[MISSING_COLUMN] Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 7 -- !query @@ -3287,7 +3287,7 @@ select uunique1 from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7 +[MISSING_COLUMN] Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7 -- !query @@ -3487,7 +3487,7 @@ select f1,g from int4_tbl a, (select f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 37 +[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 37 -- !query @@ -3496,7 +3496,7 @@ select f1,g from int4_tbl a, (select a.f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 37 +[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 37 -- !query @@ -3505,7 +3505,7 @@ select f1,g from int4_tbl a cross join (select f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 47 +[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 47 -- !query @@ -3514,7 +3514,7 @@ select f1,g from int4_tbl a cross join (select a.f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 47 +[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 47 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out index 3cd3087501bc5..600d260b094c6 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out @@ -152,7 +152,7 @@ SELECT 1 AS one FROM test_having HAVING a > 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 40 +[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 40 -- !query @@ -177,7 +177,7 @@ SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 40) == ...1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 ^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out index 6f2d45401997d..d12d17ff17b0d 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -122,7 +122,7 @@ SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'b' does not exist. Did you mean one of the following? [count(1)]; line 1 pos 61 +[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [count(1)]; line 1 pos 61 -- !query @@ -327,7 +327,7 @@ SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'b' does not exist. Did you mean one of the following? [count(a)]; line 1 pos 61 +[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [count(a)]; line 1 pos 61 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 2b4f91c50b0dc..f9d3ad69f99e8 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -65,7 +65,7 @@ select string('four: ') || 2+2 struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'four: 2' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'four: 2' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select string('four: ') || 2+2 ^^^^^^^^^^^^^^^^^^^^^^^ @@ -77,7 +77,7 @@ select 'four: ' || 2+2 struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'four: 2' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'four: 2' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 1, position 8) == select 'four: ' || 2+2 ^^^^^^^^^^^^^^^ @@ -285,7 +285,7 @@ select format_string('%0$s', 'Hello') struct<> -- !query output org.apache.spark.sql.AnalysisException -The value of parameter(s) 'strfmt' in `format_string` is invalid: expects %1$, %2$ and so on, but got %0$.; line 1 pos 7 +[INVALID_PARAMETER_VALUE] The value of parameter(s) 'strfmt' in `format_string` is invalid: expects %1$, %2$ and so on, but got %0$.; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 99b6ea78cace1..e042fc9676802 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -80,7 +80,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 39) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 39) == SQL == SELECT 1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 @@ -94,7 +94,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 37) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 37) == SQL == SELECT 1 AS two UNION SELECT 2 UNION SELECT 2 ORDER BY 1 @@ -171,7 +171,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 41) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 41) == SQL == SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 @@ -185,7 +185,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 47) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 47) == SQL == SELECT double(1.1) AS two UNION SELECT 2 UNION SELECT double(2.0) ORDER BY 1 @@ -381,7 +381,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 20) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 20) == SQL == (SELECT 1,2,3 UNION SELECT 4,5,6) INTERSECT SELECT 4,5,6 @@ -395,7 +395,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 20) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 20) == SQL == (SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) INTERSECT SELECT 4,5,6 @@ -409,7 +409,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 20) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 20) == SQL == (SELECT 1,2,3 UNION SELECT 4,5,6) EXCEPT SELECT 4,5,6 @@ -423,7 +423,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 20) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 20) == SQL == (SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) EXCEPT SELECT 4,5,6 @@ -568,7 +568,7 @@ SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'q2' does not exist. Did you mean one of the following? [int8_tbl.q1]; line 1 pos 64 +[MISSING_COLUMN] Column 'q2' does not exist. Did you mean one of the following? [int8_tbl.q1]; line 1 pos 64 -- !query @@ -728,7 +728,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'SELECT'(line 1, pos 44) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'SELECT'(line 1, pos 44) == SQL == SELECT cast('3.4' as decimal(38, 18)) UNION SELECT 'foo' diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index 91540cfbe36db..5ae683d130ee5 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -225,7 +225,7 @@ from range(9223372036854775804, 9223372036854775807) x struct<> -- !query output org.apache.spark.SparkArithmeticException -long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -235,7 +235,7 @@ from range(-9223372036854775806, -9223372036854775805) x struct<> -- !query output org.apache.spark.SparkArithmeticException -long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. +[ARITHMETIC_OVERFLOW] long overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error. -- !query @@ -462,7 +462,7 @@ window w as (order by f_numeric range between struct<> -- !query output org.apache.spark.SparkNumberFormatException -The value 'NaN' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value 'NaN' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 3, position 13) == window w as (order by f_numeric range between ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index 85be166adc449..29ef79ad0e13f 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -72,7 +72,7 @@ insert into datetimes values struct<> -- !query output org.apache.spark.sql.AnalysisException -failed to evaluate expression CAST('11:00 BST' AS TIMESTAMP): The value '11:00 BST' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +failed to evaluate expression CAST('11:00 BST' AS TIMESTAMP): [CAST_INVALID_INPUT] The value '11:00 BST' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 2, position 24) == (1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), ... ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -333,7 +333,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'BY'(line 1, pos 33) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'BY'(line 1, pos 33) == SQL == SELECT * FROM rank() OVER (ORDER BY random()) @@ -365,7 +365,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'BY': extra input 'BY'(line 1, pos 45) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'BY': extra input 'BY'(line 1, pos 45) == SQL == select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index a685214ec792b..0e7a80aa2a3c5 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -501,7 +501,7 @@ FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) struct<> -- !query output org.apache.spark.sql.AnalysisException -failed to evaluate expression CAST('nan' AS INT): The value 'nan' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +failed to evaluate expression CAST('nan' AS INT): [CAST_INVALID_INPUT] The value 'nan' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. == SQL(line 3, position 29) == FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) ^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index dbe60d88ef45f..3cdbbe475b3b0 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -36,7 +36,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +[MISSING_COLUMN] Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query @@ -45,7 +45,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't.`(a)?+.+`' does not exist. Did you mean one of the following? [t.A, t.B, t.c, t.d]; line 1 pos 7 +[MISSING_COLUMN] Column 't.`(a)?+.+`' does not exist. Did you mean one of the following? [t.A, t.B, t.c, t.d]; line 1 pos 7 -- !query @@ -54,7 +54,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a|b)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +[MISSING_COLUMN] Column '`(a|b)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query @@ -63,7 +63,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +[MISSING_COLUMN] Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query @@ -72,7 +72,7 @@ SELECT SUM(`(a|b)?+.+`) FROM testData2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 +[MISSING_COLUMN] Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 -- !query @@ -81,7 +81,7 @@ SELECT SUM(`(a)`) FROM testData2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 +[MISSING_COLUMN] Column '`(a)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 -- !query @@ -301,7 +301,7 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)` struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a)`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38 +[MISSING_COLUMN] Column '`(a)`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38 -- !query @@ -310,4 +310,4 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` struct<> -- !query output org.apache.spark.sql.AnalysisException -Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38 +[MISSING_COLUMN] Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38 diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index 70a4822ff916d..808d76b4aa45b 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -168,7 +168,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near end of input(line 1, pos 19) +[PARSE_SYNTAX_ERROR] Syntax error at or near end of input(line 1, pos 19) == SQL == SHOW TABLE EXTENDED @@ -193,7 +193,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'PARTITION'(line 1, pos 20) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'PARTITION'(line 1, pos 20) == SQL == SHOW TABLE EXTENDED PARTITION(c='Us', d=1) diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 8a61d17604175..32b7a4361c9cc 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -137,4 +137,4 @@ ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't1a' does not exist. Did you mean one of the following? [t2.t2a, t2.t2b, t2.t2c]; line 4 pos 44 +[MISSING_COLUMN] Column 't1a' does not exist. Did you mean one of the following? [t2.t2a, t2.t2b, t2.t2c]; line 4 pos 44 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 9c95b31a4fb49..d5f721479569d 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -60,7 +60,7 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a' does not exist. Did you mean one of the following? [t.c, t.d]; line 1 pos 7 +[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [t.c, t.d]; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out index 9974a26c76a65..24c78a245ca7e 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out @@ -727,7 +727,7 @@ select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -736,7 +736,7 @@ select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -745,7 +745,7 @@ select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -754,7 +754,7 @@ select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -763,7 +763,7 @@ select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat' struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -772,7 +772,7 @@ select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMM struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !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 9ba57ad8de314..49b7ff27d9b64 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 @@ -98,7 +98,7 @@ SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) struct<> -- !query output org.apache.spark.SparkDateTimeException -The fraction of sec must be zero. Valid range is [0, 60]. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[INVALID_FRACTION_OF_SECOND] The fraction of sec must be zero. Valid range is [0, 60]. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -332,7 +332,7 @@ select to_timestamp(1) struct<> -- !query output org.apache.spark.SparkDateTimeException -The value '1' of the type "STRING" cannot be cast to "TIMESTAMP_NTZ" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. +[CAST_INVALID_INPUT] The value '1' of the type "STRING" cannot be cast to "TIMESTAMP_NTZ" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. -- !query @@ -761,7 +761,7 @@ select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out index 85fa0beb99061..4831440fb2dab 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out @@ -754,7 +754,7 @@ select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/transform.sql.out b/sql/core/src/test/resources/sql-tests/results/transform.sql.out index be57390761ba3..6d00f1b68f248 100644 --- a/sql/core/src/test/resources/sql-tests/results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/transform.sql.out @@ -719,7 +719,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) +[UNSUPPORTED_FEATURE] The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) == SQL == SELECT TRANSFORM(DISTINCT b, a, c) @@ -739,7 +739,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) +[UNSUPPORTED_FEATURE] The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) == SQL == SELECT TRANSFORM(ALL b, a, c) @@ -760,7 +760,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'AS'(line 1, pos 19) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'AS'(line 1, pos 19) == SQL == SELECT TRANSFORM(b AS b_1, MAX(a), CAST(sum(c) AS STRING)) @@ -782,7 +782,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'b_1'(line 1, pos 19) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'b_1'(line 1, pos 19) == SQL == SELECT TRANSFORM(b b_1, MAX(a), CAST(sum(c) AS STRING)) @@ -804,7 +804,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Syntax error at or near 'AS'(line 1, pos 27) +[PARSE_SYNTAX_ERROR] Syntax error at or near 'AS'(line 1, pos 27) == SQL == SELECT TRANSFORM(b, MAX(a) AS max_a, CAST(sum(c) AS STRING)) diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index fd4f8b2c7a0e3..bf8966feea264 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -139,7 +139,7 @@ select to_timestamp('2018-01-01', a) from t struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -156,7 +156,7 @@ select to_unix_timestamp('2018-01-01', a) from t struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query @@ -173,7 +173,7 @@ select unix_timestamp('2018-01-01', a) from t struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +[INCONSISTENT_BEHAVIOR_CROSS_VERSION] You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index f82184933f7e9..db644af97d145 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -405,4 +405,4 @@ from tenk1 o struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 67 +[MISSING_COLUMN] Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 67 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out index 6d988bcae9b46..0365f8d8bb184 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out @@ -3276,7 +3276,7 @@ select * from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72 +[MISSING_COLUMN] Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72 -- !query @@ -3295,7 +3295,7 @@ select udf(t1.uunique1) from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11 +[MISSING_COLUMN] Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11 -- !query @@ -3305,7 +3305,7 @@ select udf(udf(t2.uunique1)) from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 15 +[MISSING_COLUMN] Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 15 -- !query @@ -3315,7 +3315,7 @@ select udf(uunique1) from struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11 +[MISSING_COLUMN] Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11 -- !query @@ -3515,7 +3515,7 @@ select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 55 +[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 55 -- !query @@ -3524,7 +3524,7 @@ select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 42 +[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 42 -- !query @@ -3533,7 +3533,7 @@ select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 61 +[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 61 -- !query @@ -3542,7 +3542,7 @@ select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 60 +[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 60 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out index bcd9bda90c316..f299dddaed755 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out @@ -152,7 +152,7 @@ SELECT 1 AS one FROM test_having HAVING udf(a) > 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 44 +[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 44 -- !query @@ -177,7 +177,7 @@ SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2 struct<> -- !query output org.apache.spark.SparkArithmeticException -Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. +[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error. == SQL(line 1, position 40) == ...1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2 ^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out index c499782a9c436..2012a60d9e927 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -125,7 +125,7 @@ SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'b' does not exist. Did you mean one of the following? [udf(count(1))]; line 1 pos 75 +[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [udf(count(1))]; line 1 pos 75 -- !query @@ -330,7 +330,7 @@ SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf( struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'b' does not exist. Did you mean one of the following? [udf(count(udf(a)))]; line 1 pos 80 +[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [udf(count(udf(a)))]; line 1 pos 80 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out index 7d2fad8cc2087..b180f70d9695f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out @@ -255,7 +255,7 @@ SELECT course, udf(year) FROM courseSales GROUP BY udf(course), year HAVING GROU struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query @@ -264,7 +264,7 @@ SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROU struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query @@ -319,7 +319,7 @@ SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GR struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query @@ -328,7 +328,7 @@ SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GR struct<> -- !query output org.apache.spark.sql.AnalysisException -grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup +[UNSUPPORTED_GROUPING_EXPRESSION] grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index d543c6a1bb742..1d2a3a63256fb 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -202,7 +202,7 @@ SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 57 +[MISSING_COLUMN] Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 57 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index 0dccf39d435f9..b03e8ded167c3 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -232,7 +232,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 +[MISSING_COLUMN] Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 -- !query @@ -313,7 +313,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct +[PIVOT_VALUE_DATA_TYPE_MISMATCH] Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct -- !query @@ -326,7 +326,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15 +[MISSING_COLUMN] Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15 -- !query @@ -339,7 +339,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Literal expressions required for pivot values, found "course". +[NON_LITERAL_PIVOT_VALUES] Literal expressions required for pivot values, found "course". -- !query @@ -424,7 +424,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid pivot column `__auto_generated_subquery_name`.`m`. Pivot columns must be comparable. +[INCOMPARABLE_PIVOT_COLUMN] Invalid pivot column `__auto_generated_subquery_name`.`m`. Pivot columns must be comparable. -- !query @@ -441,7 +441,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid pivot column `named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`. Pivot columns must be comparable. +[INCOMPARABLE_PIVOT_COLUMN] Invalid pivot column `named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`. Pivot columns must be comparable. -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 7e0fd2772d804..fb88ab7c41ba7 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -898,7 +898,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Invalid SQL syntax: The definition of window `w` is repetitive.(line 8, pos 0) +[INVALID_SQL_SYNTAX] Invalid SQL syntax: The definition of window `w` is repetitive.(line 8, pos 0) == SQL == SELECT diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 72e2d77bba624..74d5b19c3a64f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -197,23 +197,6 @@ class DataFrameAggregateSuite extends QueryTest intercept[AnalysisException] { courseSales.groupBy().agg(grouping_id("course")).explain() } - - val groupingColMismatchEx = intercept[AnalysisException] { - courseSales.cube("course", "year").agg(grouping("earnings")).explain() - } - assert(groupingColMismatchEx.getErrorClass == "GROUPING_COLUMN_MISMATCH") - assert(groupingColMismatchEx.getMessage.matches( - "Column of grouping \\(earnings.*\\) can't be found in grouping columns course.*,year.*")) - - - val groupingIdColMismatchEx = intercept[AnalysisException] { - courseSales.cube("course", "year").agg(grouping_id("earnings")).explain() - } - assert(groupingIdColMismatchEx.getErrorClass == "GROUPING_ID_COLUMN_MISMATCH") - assert(groupingIdColMismatchEx.getMessage.matches( - "Columns of grouping_id \\(earnings.*\\) does not match " + - "grouping columns \\(course.*,year.*\\)"), - groupingIdColMismatchEx.getMessage) } test("grouping/grouping_id inside window function") { @@ -1278,12 +1261,14 @@ class DataFrameAggregateSuite extends QueryTest val error = intercept[SparkException] { checkAnswer(df2.select(sum($"year-month")), Nil) } - assert(error.toString contains "SparkArithmeticException: integer overflow") + assert(error.toString contains + "SparkArithmeticException: [ARITHMETIC_OVERFLOW] integer overflow") val error2 = intercept[SparkException] { checkAnswer(df2.select(sum($"day")), Nil) } - assert(error2.toString contains "SparkArithmeticException: long overflow") + assert(error2.toString contains + "SparkArithmeticException: [ARITHMETIC_OVERFLOW] long overflow") } test("SPARK-34837: Support ANSI SQL intervals by the aggregate function `avg`") { @@ -1412,12 +1397,14 @@ class DataFrameAggregateSuite extends QueryTest val error = intercept[SparkException] { checkAnswer(df2.select(avg($"year-month")), Nil) } - assert(error.toString contains "SparkArithmeticException: integer overflow") + assert(error.toString contains + "SparkArithmeticException: [ARITHMETIC_OVERFLOW] integer overflow") val error2 = intercept[SparkException] { checkAnswer(df2.select(avg($"day")), Nil) } - assert(error2.toString contains "SparkArithmeticException: long overflow") + assert(error2.toString contains + "SparkArithmeticException: [ARITHMETIC_OVERFLOW] long overflow") val df3 = intervalData.filter($"class" > 4) val avgDF3 = df3.select(avg($"year-month"), avg($"day")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala index a8fbfc49a5086..009ba94d24462 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala @@ -19,11 +19,10 @@ package org.apache.spark.sql.errors import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.connector.{DatasourceV2SQLBase, FakeV2Provider} -import org.apache.spark.sql.test.SharedSparkSession class QueryCompilationErrorsDSv2Suite extends QueryTest - with SharedSparkSession + with QueryErrorsSuiteBase with DatasourceV2SQLBase { test("UNSUPPORTED_FEATURE: IF PARTITION NOT EXISTS not supported by INSERT") { @@ -42,10 +41,12 @@ class QueryCompilationErrorsDSv2Suite } checkAnswer(spark.table(tbl), spark.emptyDataFrame) - assert(e.getMessage === "The feature is not supported: " + - s"""IF NOT EXISTS for the table `testcat`.`ns1`.`ns2`.`tbl` by INSERT INTO.""") - assert(e.getErrorClass === "UNSUPPORTED_FEATURE") - assert(e.getSqlState === "0A000") + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = "The feature is not supported: " + + s"""IF NOT EXISTS for the table `testcat`.`ns1`.`ns2`.`tbl` by INSERT INTO.""") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 4a5890926c8ef..2c18fa927cd5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.errors import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, QueryTest} import org.apache.spark.sql.functions.{grouping, grouping_id, sum} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession case class StringLongClass(a: String, b: Long) @@ -28,35 +27,41 @@ case class StringIntClass(a: String, b: Int) case class ComplexClass(a: Long, b: StringLongClass) -class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { +class QueryCompilationErrorsSuite extends QueryTest with QueryErrorsSuiteBase { import testImplicits._ test("CANNOT_UP_CAST_DATATYPE: invalid upcast data type") { - val msg1 = intercept[AnalysisException] { + val e1 = intercept[AnalysisException] { sql("select 'value1' as a, 1L as b").as[StringIntClass] - }.message - assert(msg1 === - s""" - |Cannot up cast b from "BIGINT" to "INT". - |The type path of the target object is: - |- field (class: "scala.Int", name: "b") - |- root class: "org.apache.spark.sql.errors.StringIntClass" - |You can either add an explicit cast to the input data or choose a higher precision type + } + checkErrorClass( + exception = e1, + errorClass = "CANNOT_UP_CAST_DATATYPE", + msg = + s""" + |Cannot up cast b from "BIGINT" to "INT". + |The type path of the target object is: + |- field (class: "scala.Int", name: "b") + |- root class: "org.apache.spark.sql.errors.StringIntClass" + |You can either add an explicit cast to the input data or choose a higher precision type """.stripMargin.trim + " of the field in the target object") - val msg2 = intercept[AnalysisException] { + val e2 = intercept[AnalysisException] { sql("select 1L as a," + " named_struct('a', 'value1', 'b', cast(1.0 as decimal(38,18))) as b") .as[ComplexClass] - }.message - assert(msg2 === - s""" - |Cannot up cast b.`b` from "DECIMAL(38,18)" to "BIGINT". - |The type path of the target object is: - |- field (class: "scala.Long", name: "b") - |- field (class: "org.apache.spark.sql.errors.StringLongClass", name: "b") - |- root class: "org.apache.spark.sql.errors.ComplexClass" - |You can either add an explicit cast to the input data or choose a higher precision type + } + checkErrorClass( + exception = e2, + errorClass = "CANNOT_UP_CAST_DATATYPE", + msg = + s""" + |Cannot up cast b.`b` from "DECIMAL(38,18)" to "BIGINT". + |The type path of the target object is: + |- field (class: "scala.Long", name: "b") + |- field (class: "org.apache.spark.sql.errors.StringLongClass", name: "b") + |- root class: "org.apache.spark.sql.errors.ComplexClass" + |You can either add an explicit cast to the input data or choose a higher precision type """.stripMargin.trim + " of the field in the target object") } @@ -67,13 +72,14 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { (536363, "86123A", 6, 17851) ).toDF("InvoiceNo", "StockCode", "Quantity", "CustomerID") Seq("grouping", "grouping_id").foreach { grouping => - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { df.groupBy("CustomerId").agg(Map("Quantity" -> "max")) .filter(s"$grouping(CustomerId)=17850") } - assert(errMsg.message === - "grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup") - assert(errMsg.errorClass === Some("UNSUPPORTED_GROUPING_EXPRESSION")) + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", + msg = "grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup") } } @@ -84,13 +90,14 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { (536363, "86123A", 6, 17851) ).toDF("InvoiceNo", "StockCode", "Quantity", "CustomerID") Seq(grouping("CustomerId"), grouping_id("CustomerId")).foreach { grouping => - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { df.groupBy("CustomerId").agg(Map("Quantity" -> "max")). sort(grouping) } - assert(errMsg.errorClass === Some("UNSUPPORTED_GROUPING_EXPRESSION")) - assert(errMsg.message === - "grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup") + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", + msg = "grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup") } } @@ -99,9 +106,11 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { val e = intercept[AnalysisException] { sql("select format_string('%0$s', 'Hello')") } - assert(e.errorClass === Some("INVALID_PARAMETER_VALUE")) - assert(e.message === "The value of parameter(s) 'strfmt' in `format_string` is invalid: " + - "expects %1$, %2$ and so on, but got %0$.") + checkErrorClass( + exception = e, + errorClass = "INVALID_PARAMETER_VALUE", + msg = "The value of parameter(s) 'strfmt' in `format_string` is invalid: " + + "expects %1$, %2$ and so on, but got %0$.; line 1 pos 7") } } @@ -119,10 +128,10 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { df.groupBy("CustomerId") .agg(pandasTestUDF(df("Quantity")), sum(df("Quantity"))).collect() } - - assert(e.errorClass === Some("CANNOT_USE_MIXTURE")) - assert(e.message === - "Cannot use a mixture of aggregate function and group aggregate pandas UDF") + checkErrorClass( + exception = e, + errorClass = "CANNOT_USE_MIXTURE", + msg = "Cannot use a mixture of aggregate function and group aggregate pandas UDF") } test("UNSUPPORTED_FEATURE: Using Python UDF with unsupported join condition") { @@ -144,12 +153,12 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { df1.join( df2, pythonTestUDF(df1("CustomerID") === df2("CustomerID")), "leftouter").collect() } - - assert(e.errorClass === Some("UNSUPPORTED_FEATURE")) - assert(e.getSqlState === "0A000") - assert(e.message === - "The feature is not supported: " + - "Using PythonUDF in join condition of join type LEFT OUTER is not supported.") + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = "The feature is not supported: " + + "Using PythonUDF in join condition of join type LEFT OUTER is not supported.") } test("UNSUPPORTED_FEATURE: Using pandas UDF aggregate expression with pivot") { @@ -166,11 +175,37 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { val pandasTestUDF = TestGroupedAggPandasUDF(name = "pandas_udf") df.groupBy(df("CustomerID")).pivot(df("CustomerID")).agg(pandasTestUDF(df("Quantity"))) } + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = "The feature is not supported: " + + "Pandas UDF aggregate expressions don't support pivot.") + } - assert(e.errorClass === Some("UNSUPPORTED_FEATURE")) - assert(e.getSqlState === "0A000") - assert(e.message === - "The feature is not supported: " + - "Pandas UDF aggregate expressions don't support pivot.") + test("GROUPING_COLUMN_MISMATCH: not found the grouping column") { + val groupingColMismatchEx = intercept[AnalysisException] { + courseSales.cube("course", "year").agg(grouping("earnings")).explain() + } + checkErrorClass( + exception = groupingColMismatchEx, + errorClass = "GROUPING_COLUMN_MISMATCH", + sqlState = Some("42000"), + msg = + "Column of grouping \\(earnings.*\\) can't be found in grouping columns course.*,year.*", + matchMsg = true) + } + + test("GROUPING_ID_COLUMN_MISMATCH: columns of grouping_id does not match") { + val groupingIdColMismatchEx = intercept[AnalysisException] { + courseSales.cube("course", "year").agg(grouping_id("earnings")).explain() + } + checkErrorClass( + exception = groupingIdColMismatchEx, + errorClass = "GROUPING_ID_COLUMN_MISMATCH", + sqlState = Some("42000"), + msg = "Columns of grouping_id \\(earnings.*\\) does not match " + + "grouping columns \\(course.*,year.*\\)", + matchMsg = true) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala new file mode 100644 index 0000000000000..eb7871d5559e5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.errors + +import org.apache.spark.SparkThrowable +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.test.SharedSparkSession + +trait QueryErrorsSuiteBase extends SharedSparkSession { + def checkErrorClass( + exception: Exception with SparkThrowable, + errorClass: String, + errorSubClass: Option[String] = None, + msg: String, + sqlState: Option[String] = None, + matchMsg: Boolean = false): Unit = { + assert(exception.getErrorClass === errorClass) + sqlState.foreach(state => exception.getSqlState === state) + val fullErrorClass = if (errorSubClass.isDefined) { + errorClass + "." + errorSubClass.get + } else { + errorClass + } + if (matchMsg) { + assert(exception.getMessage.matches(s"""\\[$fullErrorClass\\] """ + msg)) + } else { + assert(exception.getMessage === s"""[$fullErrorClass] """ + msg) + } + } + + def validateParsingError( + sqlText: String, + errorClass: String, + errorSubClass: Option[String] = None, + sqlState: String, + message: String): Unit = { + val e = intercept[ParseException] { + sql(sqlText) + } + + val fullErrorClass = if (errorSubClass.isDefined) { + errorClass + "." + errorSubClass.get + } else { + errorClass + } + assert(e.getErrorClass === errorClass) + assert(e.getSqlState === sqlState) + assert(e.getMessage === s"""\n[$fullErrorClass] """ + message) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index e6ce1d7008039..3d5c123b2d936 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -17,17 +17,17 @@ package org.apache.spark.sql.errors -import org.apache.spark.{SparkArithmeticException, SparkException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException} -import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException} +import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.execution.datasources.orc.OrcTest import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions.{lit, lower, struct, sum} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.EXCEPTION -import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.util.Utils class QueryExecutionErrorsSuite extends QueryTest - with ParquetTest with OrcTest with SharedSparkSession { + with ParquetTest with OrcTest with QueryErrorsSuiteBase { import testImplicits._ @@ -51,14 +51,15 @@ class QueryExecutionErrorsSuite extends QueryTest test("INVALID_PARAMETER_VALUE: invalid key lengths in AES functions") { val (df1, df2) = getAesInputs() def checkInvalidKeyLength(df: => DataFrame): Unit = { - val e = intercept[SparkException] { - df.collect - }.getCause.asInstanceOf[SparkRuntimeException] - assert(e.getErrorClass === "INVALID_PARAMETER_VALUE") - assert(e.getSqlState === "22023") - assert(e.getMessage.matches( - "The value of parameter\\(s\\) 'key' in the `aes_encrypt`/`aes_decrypt` function " + - "is invalid: expects a binary value with 16, 24 or 32 bytes, but got \\d+ bytes.")) + checkErrorClass( + exception = intercept[SparkException] { + df.collect + }.getCause.asInstanceOf[SparkRuntimeException], + errorClass = "INVALID_PARAMETER_VALUE", + sqlState = Some("22023"), + msg = "The value of parameter\\(s\\) 'key' in the `aes_encrypt`/`aes_decrypt` function " + + "is invalid: expects a binary value with 16, 24 or 32 bytes, but got \\d+ bytes.", + matchMsg = true) } // Encryption failure - invalid key length @@ -88,13 +89,16 @@ class QueryExecutionErrorsSuite extends QueryTest val e = intercept[SparkException] { df2.selectExpr(s"aes_decrypt(unbase64($colName), binary('$key'), 'ECB')").collect }.getCause.asInstanceOf[SparkRuntimeException] - assert(e.getErrorClass === "INVALID_PARAMETER_VALUE") - assert(e.getSqlState === "22023") - assert(e.getMessage === - "The value of parameter(s) 'expr, key' in the `aes_encrypt`/`aes_decrypt` function " + - "is invalid: Detail message: " + - "Given final block not properly padded. " + - "Such issues can arise if a bad key is used during decryption.") + checkErrorClass( + exception = e, + errorClass = "INVALID_PARAMETER_VALUE", + sqlState = Some("22023"), + msg = + "The value of parameter(s) 'expr, key' in the `aes_encrypt`/`aes_decrypt` function " + + "is invalid: Detail message: " + + "Given final block not properly padded. " + + "Such issues can arise if a bad key is used during decryption." + ) } } @@ -106,10 +110,14 @@ class QueryExecutionErrorsSuite extends QueryTest val e = intercept[SparkException] { df.collect }.getCause.asInstanceOf[SparkRuntimeException] - assert(e.getErrorClass === "UNSUPPORTED_FEATURE") - assert(e.getSqlState === "0A000") - assert(e.getMessage.matches("""The feature is not supported: AES-\w+ with the padding \w+""" + - " by the `aes_encrypt`/`aes_decrypt` function.")) + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = """The feature is not supported: AES-\w+ with the padding \w+""" + + " by the `aes_encrypt`/`aes_decrypt` function.", + matchMsg = true + ) } // Unsupported AES mode and padding in encrypt @@ -125,9 +133,13 @@ class QueryExecutionErrorsSuite extends QueryTest test("UNSUPPORTED_FEATURE: unsupported types (map and struct) in lit()") { def checkUnsupportedTypeInLiteral(v: Any): Unit = { val e1 = intercept[SparkRuntimeException] { lit(v) } - assert(e1.getErrorClass === "UNSUPPORTED_FEATURE") - assert(e1.getSqlState === "0A000") - assert(e1.getMessage.matches("""The feature is not supported: literal for '.+' of .+\.""")) + checkErrorClass( + exception = e1, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = """The feature is not supported: literal for '.+' of .+\.""", + matchMsg = true + ) } checkUnsupportedTypeInLiteral(Map("key1" -> 1, "key2" -> 2)) checkUnsupportedTypeInLiteral(("mike", 29, 1.0)) @@ -139,8 +151,13 @@ class QueryExecutionErrorsSuite extends QueryTest .agg(sum($"sales.earnings")) .collect() } - assert(e2.getMessage === "The feature is not supported: pivoting by the value" + - """ '[dotnet,Dummies]' of the column data type "STRUCT".""") + checkErrorClass( + exception = e2, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = "The feature is not supported: pivoting by the value" + + """ '[dotnet,Dummies]' of the column data type "STRUCT".""" + ) } test("UNSUPPORTED_FEATURE: unsupported pivot operations") { @@ -152,9 +169,12 @@ class QueryExecutionErrorsSuite extends QueryTest .agg(sum($"sales.earnings")) .collect() } - assert(e1.getErrorClass === "UNSUPPORTED_FEATURE") - assert(e1.getSqlState === "0A000") - assert(e1.getMessage === """The feature is not supported: Repeated PIVOTs.""") + checkErrorClass( + exception = e1, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = """The feature is not supported: Repeated PIVOTs.""" + ) val e2 = intercept[SparkUnsupportedOperationException] { trainingSales @@ -163,9 +183,12 @@ class QueryExecutionErrorsSuite extends QueryTest .agg(sum($"sales.earnings")) .collect() } - assert(e2.getErrorClass === "UNSUPPORTED_FEATURE") - assert(e2.getSqlState === "0A000") - assert(e2.getMessage === """The feature is not supported: PIVOT not after a GROUP BY.""") + checkErrorClass( + exception = e2, + errorClass = "UNSUPPORTED_FEATURE", + sqlState = Some("0A000"), + msg = """The feature is not supported: PIVOT not after a GROUP BY.""" + ) } test("INCONSISTENT_BEHAVIOR_CROSS_VERSION: " + @@ -182,20 +205,22 @@ class QueryExecutionErrorsSuite extends QueryTest val format = "Parquet" val config = "\"" + SQLConf.PARQUET_REBASE_MODE_IN_READ.key + "\"" val option = "\"" + "datetimeRebaseMode" + "\"" - assert(e.getErrorClass === "INCONSISTENT_BEHAVIOR_CROSS_VERSION") - assert(e.getMessage === - "You may get a different result due to the upgrading to Spark >= 3.0: " + - s""" - |reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z - |from $format files can be ambiguous, as the files may be written by - |Spark 2.x or legacy versions of Hive, which uses a legacy hybrid calendar - |that is different from Spark 3.0+'s Proleptic Gregorian calendar. - |See more details in SPARK-31404. You can set the SQL config $config or - |the datasource option $option to "LEGACY" to rebase the datetime values - |w.r.t. the calendar difference during reading. To read the datetime values - |as it is, set the SQL config $config or the datasource option $option - |to "CORRECTED". - |""".stripMargin) + checkErrorClass( + exception = e, + errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION", + msg = + "You may get a different result due to the upgrading to Spark >= 3.0: " + + s""" + |reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z + |from $format files can be ambiguous, as the files may be written by + |Spark 2.x or legacy versions of Hive, which uses a legacy hybrid calendar + |that is different from Spark 3.0+'s Proleptic Gregorian calendar. + |See more details in SPARK-31404. You can set the SQL config $config or + |the datasource option $option to "LEGACY" to rebase the datetime values + |w.r.t. the calendar difference during reading. To read the datetime values + |as it is, set the SQL config $config or the datasource option $option + |to "CORRECTED". + |""".stripMargin) } // Fail to write ancient datetime values. @@ -208,20 +233,22 @@ class QueryExecutionErrorsSuite extends QueryTest val format = "Parquet" val config = "\"" + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key + "\"" - assert(e.getErrorClass === "INCONSISTENT_BEHAVIOR_CROSS_VERSION") - assert(e.getMessage === - "You may get a different result due to the upgrading to Spark >= 3.0: " + - s""" - |writing dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z - |into $format files can be dangerous, as the files may be read by Spark 2.x - |or legacy versions of Hive later, which uses a legacy hybrid calendar that - |is different from Spark 3.0+'s Proleptic Gregorian calendar. See more - |details in SPARK-31404. You can set $config to "LEGACY" to rebase the - |datetime values w.r.t. the calendar difference during writing, to get maximum - |interoperability. Or set $config to "CORRECTED" to write the datetime - |values as it is, if you are 100% sure that the written files will only be read by - |Spark 3.0+ or other systems that use Proleptic Gregorian calendar. - |""".stripMargin) + checkErrorClass( + exception = e, + errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION", + msg = + "You may get a different result due to the upgrading to Spark >= 3.0: " + + s""" + |writing dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z + |into $format files can be dangerous, as the files may be read by Spark 2.x + |or legacy versions of Hive later, which uses a legacy hybrid calendar that + |is different from Spark 3.0+'s Proleptic Gregorian calendar. See more + |details in SPARK-31404. You can set $config to "LEGACY" to rebase the + |datetime values w.r.t. the calendar difference during writing, to get maximum + |interoperability. Or set $config to "CORRECTED" to write the datetime + |values as it is, if you are 100% sure that the written files will only be read by + |Spark 3.0+ or other systems that use Proleptic Gregorian calendar. + |""".stripMargin) } } } @@ -233,10 +260,11 @@ class QueryExecutionErrorsSuite extends QueryTest val e = intercept[SparkException] { spark.read.schema("time timestamp_ntz").orc(file.getCanonicalPath).collect() }.getCause.asInstanceOf[SparkUnsupportedOperationException] - - assert(e.getErrorClass === "UNSUPPORTED_OPERATION") - assert(e.getMessage === "The operation is not supported: " + - "Unable to convert \"TIMESTAMP\" of Orc to data type \"TIMESTAMP_NTZ\".") + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_OPERATION", + msg = "The operation is not supported: " + + "Unable to convert \"TIMESTAMP\" of Orc to data type \"TIMESTAMP_NTZ\".") } } } @@ -248,10 +276,11 @@ class QueryExecutionErrorsSuite extends QueryTest val e = intercept[SparkException] { spark.read.schema("time timestamp_ltz").orc(file.getCanonicalPath).collect() }.getCause.asInstanceOf[SparkUnsupportedOperationException] - - assert(e.getErrorClass === "UNSUPPORTED_OPERATION") - assert(e.getMessage === "The operation is not supported: " + - "Unable to convert \"TIMESTAMP_NTZ\" of Orc to data type \"TIMESTAMP\".") + checkErrorClass( + exception = e, + errorClass = "UNSUPPORTED_OPERATION", + msg = "The operation is not supported: " + + "Unable to convert \"TIMESTAMP_NTZ\" of Orc to data type \"TIMESTAMP\".") } } } @@ -260,9 +289,36 @@ class QueryExecutionErrorsSuite extends QueryTest val e = intercept[SparkArithmeticException] { sql("select timestampadd(YEAR, 1000000, timestamp'2022-03-09 01:02:03')").collect() } - assert(e.getErrorClass === "DATETIME_OVERFLOW") - assert(e.getSqlState === "22008") - assert(e.getMessage === - "Datetime operation overflow: add 1000000 YEAR to TIMESTAMP '2022-03-09 01:02:03'.") + checkErrorClass( + exception = e, + errorClass = "DATETIME_OVERFLOW", + sqlState = Some("22008"), + msg = "Datetime operation overflow: add 1000000 YEAR to TIMESTAMP '2022-03-09 01:02:03'.") + } + + test("UNSUPPORTED_SAVE_MODE: unsupported null saveMode whether the path exists or not") { + withTempPath { path => + val e1 = intercept[SparkIllegalArgumentException] { + val saveMode: SaveMode = null + Seq(1, 2).toDS().write.mode(saveMode).parquet(path.getAbsolutePath) + } + checkErrorClass( + exception = e1, + errorClass = "UNSUPPORTED_SAVE_MODE", + errorSubClass = Some("NON_EXISTENT_PATH"), + msg = "The save mode NULL is not supported for: a not existent path.") + + Utils.createDirectory(path) + + val e2 = intercept[SparkIllegalArgumentException] { + val saveMode: SaveMode = null + Seq(1, 2).toDS().write.mode(saveMode).parquet(path.getAbsolutePath) + } + checkErrorClass( + exception = e2, + errorClass = "UNSUPPORTED_SAVE_MODE", + errorSubClass = Some("EXISTENT_PATH"), + msg = "The save mode NULL is not supported for: an existent path.") + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 508f551bcec95..575430404a57a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -18,24 +18,10 @@ package org.apache.spark.sql.errors import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.test.SharedSparkSession // Turn of the length check because most of the tests check entire error messages // scalastyle:off line.size.limit -class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { - def validateParsingError( - sqlText: String, - errorClass: String, - sqlState: String, - message: String): Unit = { - val e = intercept[ParseException] { - sql(sqlText) - } - assert(e.getErrorClass === errorClass) - assert(e.getSqlState === sqlState) - assert(e.getMessage === message) - } +class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { test("UNSUPPORTED_FEATURE: LATERAL join with NATURAL join not supported") { validateParsingError( @@ -43,8 +29,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """ - |The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14) + """The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14) | |== SQL == |SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) @@ -58,8 +43,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """ - |The feature is not supported: LATERAL join with USING join.(line 1, pos 14) + """The feature is not supported: LATERAL join with USING join.(line 1, pos 14) | |== SQL == |SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) @@ -74,8 +58,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - s""" - |The feature is not supported: LATERAL join type $joinType.(line 1, pos 14) + s"""The feature is not supported: LATERAL join type $joinType.(line 1, pos 14) | |== SQL == |SELECT * FROM t1 $joinType JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 @@ -98,8 +81,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - s""" - |Invalid SQL syntax: LATERAL can only be used with subquery.(line 1, pos $pos) + s"""Invalid SQL syntax: LATERAL can only be used with subquery.(line 1, pos $pos) | |== SQL == |$sqlText @@ -114,8 +96,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """ - |The feature is not supported: NATURAL CROSS JOIN.(line 1, pos 14) + """The feature is not supported: NATURAL CROSS JOIN.(line 1, pos 14) | |== SQL == |SELECT * FROM a NATURAL CROSS JOIN b @@ -129,8 +110,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - """ - |Invalid SQL syntax: The definition of window `win` is repetitive.(line 1, pos 31) + """Invalid SQL syntax: The definition of window `win` is repetitive.(line 1, pos 31) | |== SQL == |SELECT min(a) OVER win FROM t1 WINDOW win AS win, win AS win2 @@ -144,8 +124,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - """ - |Invalid SQL syntax: Window reference `win` is not a window specification.(line 1, pos 31) + """Invalid SQL syntax: Window reference `win` is not a window specification.(line 1, pos 31) | |== SQL == |SELECT min(a) OVER win FROM t1 WINDOW win AS win @@ -159,8 +138,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - """ - |Invalid SQL syntax: Cannot resolve window reference `win2`.(line 1, pos 31) + """Invalid SQL syntax: Cannot resolve window reference `win2`.(line 1, pos 31) | |== SQL == |SELECT min(a) OVER win FROM t1 WINDOW win AS win2 @@ -174,8 +152,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """ - |The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) + """The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) | |== SQL == |SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t @@ -190,8 +167,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """ - |The feature is not supported: TRANSFORM with serde is only supported in hive mode(line 1, pos 0) + """The feature is not supported: TRANSFORM with serde is only supported in hive mode(line 1, pos 0) | |== SQL == |SELECT TRANSFORM(a) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' USING 'a' FROM t @@ -205,8 +181,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - """ - |Invalid SQL syntax: Too many arguments for transform `years`(line 1, pos 44) + """Invalid SQL syntax: Too many arguments for transform `years`(line 1, pos 44) | |== SQL == |CREATE TABLE table(col int) PARTITIONED BY (years(col,col))