diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 26d75fa675e23..a105b5effc16b 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -106,7 +106,7 @@ "sqlState" : "22023" }, "INVALID_INPUT_SYNTAX_FOR_NUMERIC_TYPE" : { - "message" : [ "invalid input syntax for type numeric: %s. To return NULL instead, use 'try_cast'. If necessary set %s to false to bypass this error.%s" ], + "message" : [ "invalid input syntax for type %s: %s. To return NULL instead, use 'try_cast'. If necessary set %s to false to bypass this error.%s" ], "sqlState" : "42000" }, "INVALID_JSON_SCHEMA_MAPTYPE" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index e522c211cb228..f17e8d8194a76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -816,7 +816,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit }) case StringType if ansiEnabled => buildCast[UTF8String](_, - s => changePrecision(Decimal.fromStringANSI(s, origin.context), target)) + s => changePrecision(Decimal.fromStringANSI(s, target.sql, origin.context), target)) case BooleanType => buildCast[Boolean](_, b => toPrecision(if (b) Decimal.ONE else Decimal.ZERO, target)) case DateType => @@ -845,7 +845,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case _: NumberFormatException => val d = Cast.processFloatingPointSpecialLiterals(doubleStr, false) if(ansiEnabled && d == null) { - throw QueryExecutionErrors.invalidInputSyntaxForNumericError(s, origin.context) + throw QueryExecutionErrors.invalidInputSyntaxForNumericError( + s, DoubleType.sql, origin.context) } else { d } @@ -870,7 +871,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case _: NumberFormatException => val f = Cast.processFloatingPointSpecialLiterals(floatStr, true) if (ansiEnabled && f == null) { - throw QueryExecutionErrors.invalidInputSyntaxForNumericError(s, origin.context) + throw QueryExecutionErrors.invalidInputSyntaxForNumericError( + s, FloatType.sql, origin.context) } else { f } @@ -1376,9 +1378,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit """ case StringType if ansiEnabled => val errorContext = ctx.addReferenceObj("errCtx", origin.context) + val decimalType = "\"" + target.sql + "\"" (c, evPrim, evNull) => code""" - Decimal $tmp = Decimal.fromStringANSI($c, $errorContext); + Decimal $tmp = Decimal.fromStringANSI($c, $decimalType, $errorContext); ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast, ctx)} """ case BooleanType => @@ -1896,10 +1899,12 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit from match { case StringType => val floatStr = ctx.freshVariable("floatStr", StringType) + val targetType = "\"" + FloatType.sql + "\"" (c, evPrim, evNull) => val handleNull = if (ansiEnabled) { val errorContext = ctx.addReferenceObj("errCtx", origin.context) - s"throw QueryExecutionErrors.invalidInputSyntaxForNumericError($c, $errorContext);" + "throw QueryExecutionErrors.invalidInputSyntaxForNumericError(" + + s"$c, $targetType, $errorContext);" } else { s"$evNull = true;" } @@ -1936,7 +1941,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (c, evPrim, evNull) => val handleNull = if (ansiEnabled) { val errorContext = ctx.addReferenceObj("errCtx", origin.context) - s"throw QueryExecutionErrors.invalidInputSyntaxForNumericError($c, $errorContext);" + val targetType = "\"" + DoubleType.sql + "\"" + "throw QueryExecutionErrors.invalidInputSyntaxForNumericError(" + + s"$c, $targetType, $errorContext);" } else { s"$evNull = true;" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UTF8StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UTF8StringUtils.scala index 9589cf3774ee1..61cfa1e6249f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UTF8StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UTF8StringUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types.{ByteType, IntegerType, LongType, ShortType} import org.apache.spark.unsafe.types.UTF8String /** @@ -26,23 +27,27 @@ import org.apache.spark.unsafe.types.UTF8String object UTF8StringUtils { def toLongExact(s: UTF8String, errorContext: String): Long = - withException(s.toLongExact, errorContext) + withException(s.toLongExact, s, LongType.sql, errorContext) def toIntExact(s: UTF8String, errorContext: String): Int = - withException(s.toIntExact, errorContext) + withException(s.toIntExact, s, IntegerType.sql, errorContext) def toShortExact(s: UTF8String, errorContext: String): Short = - withException(s.toShortExact, errorContext) + withException(s.toShortExact, s, ShortType.sql, errorContext) def toByteExact(s: UTF8String, errorContext: String): Byte = - withException(s.toByteExact, errorContext) + withException(s.toByteExact, s, ByteType.sql, errorContext) - private def withException[A](f: => A, errorContext: String): A = { + private def withException[A]( + f: => A, + s: UTF8String, + targetType: String, + errorContext: String): A = { try { f } catch { - case e: NumberFormatException => - throw QueryExecutionErrors.invalidInputSyntaxForNumericError(e, errorContext) + case _: NumberFormatException => + throw QueryExecutionErrors.invalidInputSyntaxForNumericError(s, targetType, errorContext) } } } 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 79e36cb485a97..1c51ef8c7c03a 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 @@ -104,18 +104,13 @@ object QueryExecutionErrors extends QueryErrorsBase { decimalPrecision.toString, decimalScale.toString, SQLConf.ANSI_ENABLED.key, context)) } - def invalidInputSyntaxForNumericError( - e: NumberFormatException, - errorContext: String): NumberFormatException = { - new NumberFormatException(s"${e.getMessage}. To return NULL instead, use 'try_cast'. " + - s"If necessary set ${SQLConf.ANSI_ENABLED.key} to false to bypass this error." + errorContext) - } - def invalidInputSyntaxForNumericError( s: UTF8String, + targetType: String, errorContext: String): NumberFormatException = { new SparkNumberFormatException(errorClass = "INVALID_INPUT_SYNTAX_FOR_NUMERIC_TYPE", - messageParameters = Array(toSQLValue(s, StringType), SQLConf.ANSI_ENABLED.key, errorContext)) + messageParameters = + Array(targetType, toSQLValue(s, StringType), SQLConf.ANSI_ENABLED.key, errorContext)) } def cannotCastFromNullTypeError(to: DataType): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index ac6ac33451cdf..14abbe147833d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -613,7 +613,10 @@ object Decimal { } } - def fromStringANSI(str: UTF8String, errorContext: String = ""): Decimal = { + def fromStringANSI( + str: UTF8String, + decimalType: String = "DECIMAL", + errorContext: String = ""): Decimal = { try { val bigDecimal = stringToJavaBigDecimal(str) // We fast fail because constructing a very large JavaBigDecimal to Decimal is very slow. @@ -626,7 +629,7 @@ object Decimal { } } catch { case _: NumberFormatException => - throw QueryExecutionErrors.invalidInputSyntaxForNumericError(str, errorContext) + throw QueryExecutionErrors.invalidInputSyntaxForNumericError(str, decimalType, errorContext) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala index 785fd95692e9b..8a41f3aece602 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp import java.time.DateTimeException -import org.apache.spark.SparkArithmeticException +import org.apache.spark.{SparkArithmeticException, SparkNumberFormatException} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND @@ -174,29 +174,35 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { test("cast from invalid string to numeric should throw NumberFormatException") { // cast to IntegerType Seq(IntegerType, ShortType, ByteType, LongType).foreach { dataType => + val typeName = dataType.sql + checkExceptionInExpression[SparkNumberFormatException]( + cast("string", dataType), s"invalid input syntax for type $typeName: 'string'") checkExceptionInExpression[NumberFormatException]( - cast("string", dataType), "invalid input syntax for type numeric: 'string'") + cast("123-string", dataType), s"invalid input syntax for type $typeName: '123-string'") checkExceptionInExpression[NumberFormatException]( - cast("123-string", dataType), "invalid input syntax for type numeric: '123-string'") + cast("2020-07-19", dataType), s"invalid input syntax for type $typeName: '2020-07-19'") checkExceptionInExpression[NumberFormatException]( - cast("2020-07-19", dataType), "invalid input syntax for type numeric: '2020-07-19'") - checkExceptionInExpression[NumberFormatException]( - cast("1.23", dataType), "invalid input syntax for type numeric: '1.23'") + cast("1.23", dataType), s"invalid input syntax for type $typeName: '1.23'") } Seq(DoubleType, FloatType, DecimalType.USER_DEFAULT).foreach { dataType => + val typeName = dataType.sql checkExceptionInExpression[NumberFormatException]( - cast("string", dataType), "invalid input syntax for type numeric: 'string'") + cast("string", dataType), s"invalid input syntax for type $typeName: 'string'") checkExceptionInExpression[NumberFormatException]( - cast("123.000.00", dataType), "invalid input syntax for type numeric: '123.000.00'") + cast("123.000.00", dataType), s"invalid input syntax for type $typeName: '123.000.00'") checkExceptionInExpression[NumberFormatException]( - cast("abc.com", dataType), "invalid input syntax for type numeric: 'abc.com'") + cast("abc.com", dataType), s"invalid input syntax for type $typeName: 'abc.com'") } } protected def checkCastToNumericError(l: Literal, to: DataType, tryCastResult: Any): Unit = { + val typeName = to match { + case a: ArrayType => a.elementType.sql + case _ => to.sql + } checkExceptionInExpression[NumberFormatException]( - cast(l, to), "invalid input syntax for type numeric: 'true'") + cast(l, to), s"invalid input syntax for type $typeName: 'true'") } test("cast from invalid string array to numeric array should throw NumberFormatException") { @@ -243,7 +249,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { checkExceptionInExpression[NumberFormatException]( cast("abcd", DecimalType(38, 1)), - "invalid input syntax for type numeric") + "invalid input syntax for type DECIMAL(38,1): 'abcd'") } protected def checkCastToBooleanError(l: Literal, to: DataType, tryCastResult: Any): Unit = { @@ -368,8 +374,8 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { val ret = cast(map, MapType(IntegerType, StringType, valueContainsNull = true)) assert(ret.resolved == !isTryCast) if (!isTryCast) { - checkExceptionInExpression[NumberFormatException]( - ret, "invalid input syntax for type numeric") + checkExceptionInExpression[SparkNumberFormatException]( + ret, "invalid input syntax for type INT: 'a'") } } @@ -387,7 +393,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { assert(ret.resolved == !isTryCast) if (!isTryCast) { checkExceptionInExpression[NumberFormatException]( - ret, "invalid input syntax for type numeric") + ret, "invalid input syntax for type INT: 'a'") } } } @@ -511,8 +517,8 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { assert(ret.resolved === !isTryCast) if (!isTryCast) { - checkExceptionInExpression[NumberFormatException]( - ret, "invalid input syntax for type numeric") + checkExceptionInExpression[SparkNumberFormatException]( + ret, "invalid input syntax for type INT") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index 5433c561a0379..ca9d9eea6b54d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -284,7 +284,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper assert(Decimal.fromString(UTF8String.fromString("str")) === null) val e = intercept[NumberFormatException](Decimal.fromStringANSI(UTF8String.fromString("str"))) - assert(e.getMessage.contains("invalid input syntax for type numeric")) + assert(e.getMessage.contains("invalid input syntax for type DECIMAL")) } test("SPARK-35841: Casting string to decimal type doesn't work " + diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index e391c31690fd7..4610716902e5d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -4,9 +4,11 @@ SELECT CAST('1.23' AS long); SELECT CAST('-4.56' AS int); SELECT CAST('-4.56' AS long); --- cast string which are not numbers to integral should return null +-- cast string which are not numbers to numeric types SELECT CAST('abc' AS int); SELECT CAST('abc' AS long); +SELECT CAST('abc' AS float); +SELECT CAST('abc' AS double); -- cast string representing a very large number to integral should return null SELECT CAST('1234567890123' AS int); @@ -15,14 +17,18 @@ SELECT CAST('12345678901234567890123' AS long); -- cast empty string to integral should return null SELECT CAST('' AS int); SELECT CAST('' AS long); +SELECT CAST('' AS float); +SELECT CAST('' AS double); -- cast null to integral should return null SELECT CAST(NULL AS int); SELECT CAST(NULL AS long); --- cast invalid decimal string to integral should return null +-- cast invalid decimal string to numeric types SELECT CAST('123.a' AS int); SELECT CAST('123.a' AS long); +SELECT CAST('123.a' AS float); +SELECT CAST('123.a' AS double); -- '-2147483648' is the smallest int value SELECT CAST('-2147483648' AS int); 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 3de9c1f743def..06201ef308e71 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 66 +-- Number of queries: 72 -- !query @@ -7,8 +7,8 @@ SELECT CAST('1.23' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1.23'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '1.23'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('1.23' AS int) ^^^^^^^^^^^^^^^^^^^ @@ -19,8 +19,8 @@ SELECT CAST('1.23' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1.23'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '1.23'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('1.23' AS long) ^^^^^^^^^^^^^^^^^^^^ @@ -31,8 +31,8 @@ SELECT CAST('-4.56' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '-4.56'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '-4.56'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('-4.56' AS int) ^^^^^^^^^^^^^^^^^^^^ @@ -43,8 +43,8 @@ SELECT CAST('-4.56' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '-4.56'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '-4.56'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('-4.56' AS long) ^^^^^^^^^^^^^^^^^^^^^ @@ -55,8 +55,8 @@ SELECT CAST('abc' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'abc'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: 'abc'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('abc' AS int) ^^^^^^^^^^^^^^^^^^ @@ -67,20 +67,44 @@ SELECT CAST('abc' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'abc'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: 'abc'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('abc' AS long) ^^^^^^^^^^^^^^^^^^^ +-- !query +SELECT CAST('abc' AS float) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +invalid input syntax for type FLOAT: 'abc'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +== SQL(line 1, position 7) == +SELECT CAST('abc' AS float) + ^^^^^^^^^^^^^^^^^^^^ + + +-- !query +SELECT CAST('abc' AS double) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +invalid input syntax for type DOUBLE: 'abc'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +== SQL(line 1, position 7) == +SELECT CAST('abc' AS double) + ^^^^^^^^^^^^^^^^^^^^^ + + -- !query SELECT CAST('1234567890123' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1234567890123'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '1234567890123'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('1234567890123' AS int) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -91,8 +115,8 @@ SELECT CAST('12345678901234567890123' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '12345678901234567890123'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '12345678901234567890123'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('12345678901234567890123' AS long) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -103,8 +127,8 @@ SELECT CAST('' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: ''. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: ''. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('' AS int) ^^^^^^^^^^^^^^^ @@ -115,13 +139,37 @@ SELECT CAST('' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: ''. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: ''. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('' AS long) ^^^^^^^^^^^^^^^^ +-- !query +SELECT CAST('' AS float) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +invalid input syntax for type FLOAT: ''. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +== SQL(line 1, position 7) == +SELECT CAST('' AS float) + ^^^^^^^^^^^^^^^^^ + + +-- !query +SELECT CAST('' AS double) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +invalid input syntax for type DOUBLE: ''. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +== SQL(line 1, position 7) == +SELECT CAST('' AS double) + ^^^^^^^^^^^^^^^^^^ + + -- !query SELECT CAST(NULL AS int) -- !query schema @@ -143,8 +191,8 @@ SELECT CAST('123.a' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '123.a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '123.a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('123.a' AS int) ^^^^^^^^^^^^^^^^^^^^ @@ -155,13 +203,37 @@ SELECT CAST('123.a' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '123.a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '123.a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('123.a' AS long) ^^^^^^^^^^^^^^^^^^^^^ +-- !query +SELECT CAST('123.a' AS float) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +invalid input syntax for type FLOAT: '123.a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +== SQL(line 1, position 7) == +SELECT CAST('123.a' AS float) + ^^^^^^^^^^^^^^^^^^^^^^ + + +-- !query +SELECT CAST('123.a' AS double) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +invalid input syntax for type DOUBLE: '123.a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +== SQL(line 1, position 7) == +SELECT CAST('123.a' AS double) + ^^^^^^^^^^^^^^^^^^^^^^^ + + -- !query SELECT CAST('-2147483648' AS int) -- !query schema @@ -175,8 +247,8 @@ SELECT CAST('-2147483649' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '-2147483649'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '-2147483649'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('-2147483649' AS int) ^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -195,8 +267,8 @@ SELECT CAST('2147483648' AS int) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '2147483648'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '2147483648'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('2147483648' AS int) ^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -215,8 +287,8 @@ SELECT CAST('-9223372036854775809' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '-9223372036854775809'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '-9223372036854775809'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('-9223372036854775809' AS long) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -235,8 +307,8 @@ SELECT CAST('9223372036854775808' AS long) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '9223372036854775808'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '9223372036854775808'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT CAST('9223372036854775808' AS long) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -494,8 +566,8 @@ select cast('1中文' as tinyint) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type TINYINT: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select cast('1中文' as tinyint) ^^^^^^^^^^^^^^^^^^^^^^ @@ -506,8 +578,8 @@ select cast('1中文' as smallint) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type SMALLINT: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select cast('1中文' as smallint) ^^^^^^^^^^^^^^^^^^^^^^^ @@ -518,8 +590,8 @@ select cast('1中文' as INT) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select cast('1中文' as INT) ^^^^^^^^^^^^^^^^^^ @@ -530,8 +602,8 @@ select cast('中文1' as bigint) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '中文1'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '中文1'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select cast('中文1' as bigint) ^^^^^^^^^^^^^^^^^^^^^ @@ -542,8 +614,8 @@ select cast('1中文' as bigint) -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: '1中文'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select cast('1中文' as bigint) ^^^^^^^^^^^^^^^^^^^^^ @@ -606,7 +678,7 @@ select cast('xyz' as decimal(4, 2)) struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'xyz'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DECIMAL(4,2): 'xyz'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select cast('xyz' as decimal(4, 2)) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ 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 d9777b53d21a7..77dec199b4a1b 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 @@ -326,8 +326,8 @@ select date_add('2011-11-11', '1.2') -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1.2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '1.2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select date_add('2011-11-11', '1.2') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -437,8 +437,8 @@ select date_sub(date'2011-11-11', '1.2') -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: '1.2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: '1.2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select date_sub(date'2011-11-11', '1.2') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ 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 3b8d95bca0ad2..fe08ef8abdb5d 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 -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select interval 2 second * 'a' ^^^^^^^^^^^^^^^^^^^^^^^ @@ -134,7 +134,7 @@ select interval 2 second / 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select interval 2 second / 'a' ^^^^^^^^^^^^^^^^^^^^^^^ @@ -146,7 +146,7 @@ select interval 2 year * 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select interval 2 year * 'a' ^^^^^^^^^^^^^^^^^^^^^ @@ -158,7 +158,7 @@ select interval 2 year / 'a' struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select interval 2 year / 'a' ^^^^^^^^^^^^^^^^^^^^^ @@ -186,7 +186,7 @@ select 'a' * interval 2 second struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select 'a' * interval 2 second ^^^^^^^^^^^^^^^^^^^^^^^ @@ -198,7 +198,7 @@ select 'a' * interval 2 year struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select 'a' * interval 2 year ^^^^^^^^^^^^^^^^^^^^^ 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 7d07282ab6763..7eac9d541f714 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 @@ -81,8 +81,8 @@ select left("abcd", -2), left("abcd", 0), left("abcd", 'a') -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 42) == ...t("abcd", -2), left("abcd", 0), left("abcd", 'a') ^^^^^^^^^^^^^^^^^ @@ -109,8 +109,8 @@ select right("abcd", -2), right("abcd", 0), right("abcd", 'a') -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 44) == ...("abcd", -2), right("abcd", 0), right("abcd", 'a') ^^^^^^^^^^^^^^^^^^ @@ -418,8 +418,8 @@ SELECT lpad('hi', 'invalid_length') -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'invalid_length'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: 'invalid_length'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT lpad('hi', 'invalid_length') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -430,8 +430,8 @@ SELECT rpad('hi', 'invalid_length') -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'invalid_length'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: 'invalid_length'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT rpad('hi', 'invalid_length') ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 9ed02e3bed2c6..aaa82e4351351 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 66 +-- Number of queries: 72 -- !query @@ -50,6 +50,22 @@ struct NULL +-- !query +SELECT CAST('abc' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS double) +-- !query schema +struct +-- !query output +NULL + + -- !query SELECT CAST('1234567890123' AS int) -- !query schema @@ -82,6 +98,22 @@ struct NULL +-- !query +SELECT CAST('' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS double) +-- !query schema +struct +-- !query output +NULL + + -- !query SELECT CAST(NULL AS int) -- !query schema @@ -114,6 +146,22 @@ struct NULL +-- !query +SELECT CAST('123.a' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS double) +-- !query schema +struct +-- !query output +NULL + + -- !query SELECT CAST('-2147483648' AS int) -- !query schema 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 a7ee7400e58bd..70777f8089bb8 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 -invalid input syntax for type numeric: 'N A N'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type FLOAT: 'N A N'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT float('N A N') ^^^^^^^^^^^^^^ @@ -108,7 +108,7 @@ SELECT float('NaN x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'NaN x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type FLOAT: 'NaN x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT float('NaN x') ^^^^^^^^^^^^^^ @@ -120,7 +120,7 @@ SELECT float(' INFINITY x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: ' INFINITY x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type FLOAT: ' INFINITY x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT float(' INFINITY x') ^^^^^^^^^^^^^^^^^^^^^^^ @@ -156,7 +156,7 @@ SELECT float(decimal('nan')) struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'nan'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DECIMAL(10,0): 'nan'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 13) == SELECT float(decimal('nan')) ^^^^^^^^^^^^^^ 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 3237969ea8736..9790a0fd1e103 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 -invalid input syntax for type numeric: 'N A N'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'N A N'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT double('N A N') ^^^^^^^^^^^^^^^ @@ -140,7 +140,7 @@ SELECT double('NaN x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'NaN x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: 'NaN x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT double('NaN x') ^^^^^^^^^^^^^^^ @@ -152,7 +152,7 @@ SELECT double(' INFINITY x') struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: ' INFINITY x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DOUBLE: ' INFINITY x'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == SELECT double(' INFINITY x') ^^^^^^^^^^^^^^^^^^^^^^^^ @@ -188,7 +188,7 @@ SELECT double(decimal('nan')) struct<> -- !query output org.apache.spark.SparkNumberFormatException -invalid input syntax for type numeric: 'nan'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +invalid input syntax for type DECIMAL(10,0): 'nan'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 14) == SELECT double(decimal('nan')) ^^^^^^^^^^^^^^ 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 836370935f64d..3c9fd32bfcbbe 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 @@ -64,8 +64,8 @@ select string('four: ') || 2+2 -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'four: 2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: 'four: 2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select string('four: ') || 2+2 ^^^^^^^^^^^^^^^^^^^^^^^ @@ -76,8 +76,8 @@ select 'four: ' || 2+2 -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'four: 2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type BIGINT: 'four: 2'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 1, position 7) == select 'four: ' || 2+2 ^^^^^^^^^^^^^^^ 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 1d48d7c7b92c4..d56d2f072716b 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 @@ -461,8 +461,8 @@ window w as (order by f_numeric range between -- !query schema struct<> -- !query output -java.lang.NumberFormatException -invalid input syntax for type numeric: 'NaN'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +org.apache.spark.SparkNumberFormatException +invalid input syntax for type INT: 'NaN'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 3, position 12) == window w as (order by f_numeric range between ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ 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 87beeacc0bc15..5c6cc18afd380 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): invalid input syntax for type numeric: 'nan'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +failed to evaluate expression CAST('nan' AS INT): invalid input syntax for type INT: 'nan'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. == SQL(line 3, position 28) == 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/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index 3ff526bd9db57..189e1b74345c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -314,7 +314,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { val errorMsg = intercept[NumberFormatException] { sql("insert into t partition(a='ansi') values('ansi')") }.getMessage - assert(errorMsg.contains("invalid input syntax for type numeric: 'ansi'")) + assert(errorMsg.contains("invalid input syntax for type INT: 'ansi'")) } else { sql("insert into t partition(a='ansi') values('ansi')") checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil)