From fb74350ec92b5800e1305b0cc7c061410e986ea7 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 11:55:54 +0300 Subject: [PATCH 1/8] Add toSQLType --- .../scala/org/apache/spark/sql/errors/QueryErrorsBase.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala index e69e1382ecf62..fa57f3c8a0014 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.errors +import java.util.Locale + import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{DataType, DoubleType, FloatType} @@ -44,4 +46,8 @@ trait QueryErrorsBase { def toSQLValue(v: Any, t: DataType): String = { litToErrorValue(Literal.create(v, t)) } + + def toSQLType(t: DataType): String = { + t.typeName.toUpperCase(Locale.ROOT) + } } From ad585863d88e86673dd3e28bf300b816eebe03e8 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 11:57:20 +0300 Subject: [PATCH 2/8] Add toSQLType --- .../scala/org/apache/spark/sql/errors/QueryErrorsBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala index fa57f3c8a0014..4fe758776721a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala @@ -48,6 +48,6 @@ trait QueryErrorsBase { } def toSQLType(t: DataType): String = { - t.typeName.toUpperCase(Locale.ROOT) + t.sql } } From 6321e54ac5515a56165e9f013e8d8d2a53d90ae7 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 12:14:13 +0300 Subject: [PATCH 3/8] Adjust QueryExecutionErrorsSuite --- .../spark/sql/errors/QueryErrorsBase.scala | 2 -- .../sql/errors/QueryExecutionErrors.scala | 18 ++++++++++-------- .../sql/errors/QueryExecutionErrorsSuite.scala | 10 +++++----- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala index 4fe758776721a..7002f19f9fc84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.errors -import java.util.Locale - import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{DataType, DoubleType, FloatType} 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 e259103382e53..30c4c5b237f94 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 @@ -91,7 +91,7 @@ object QueryExecutionErrors extends QueryErrorsBase { def castingCauseOverflowError(t: Any, dataType: DataType): ArithmeticException = { new SparkArithmeticException(errorClass = "CAST_CAUSES_OVERFLOW", - messageParameters = Array(toSQLValue(t), dataType.catalogString, SQLConf.ANSI_ENABLED.key)) + messageParameters = Array(toSQLValue(t), toSQLType(dataType), SQLConf.ANSI_ENABLED.key)) } def cannotChangeDecimalPrecisionError( @@ -244,8 +244,7 @@ object QueryExecutionErrors extends QueryErrorsBase { new SparkRuntimeException( errorClass = "UNSUPPORTED_FEATURE", messageParameters = Array( - s"pivoting by the value '${v.toString}' of the column data type" + - s" '${dataType.catalogString}'.")) + s"pivoting by the value '${v.toString}' of the column data type ${toSQLType(dataType)}.")) } def noDefaultForDataTypeError(dataType: DataType): RuntimeException = { @@ -1608,8 +1607,8 @@ object QueryExecutionErrors extends QueryErrorsBase { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_OPERATION", messageParameters = Array( - s"${TimestampType.catalogString} must supply timeZoneId parameter " + - s"while converting to ArrowType") + s"${toSQLType(TimestampType)} must supply timeZoneId parameter " + + s"while converting to the arrow timestamp type.") ) } @@ -1926,14 +1925,17 @@ object QueryExecutionErrors extends QueryErrorsBase { def cannotConvertOrcTimestampToTimestampNTZError(): Throwable = { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_OPERATION", - messageParameters = Array("Unable to convert timestamp of Orc to data type 'timestamp_ntz'")) + messageParameters = Array( + s"Unable to convert ${toSQLType(TimestampType)} of Orc to " + + s"data type ${toSQLType(TimestampNTZType)}.")) } def cannotConvertOrcTimestampNTZToTimestampLTZError(): Throwable = { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_OPERATION", - messageParameters = - Array("Unable to convert timestamp ntz of Orc to data type 'timestamp_ltz'")) + messageParameters = Array( + s"Unable to convert ${toSQLType(TimestampNTZType)} of Orc to " + + s"data type ${toSQLType(TimestampType)}.")) } def writePartitionExceedConfigSizeWhenDynamicPartitionError( 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 09f655431dcb3..2579a7bc8e998 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 @@ -149,7 +149,7 @@ class QueryExecutionErrorsSuite extends QueryTest .collect() } assert(e2.getMessage === "The feature is not supported: pivoting by the value" + - """ '[dotnet,Dummies]' of the column data type 'struct'.""") + """ '[dotnet,Dummies]' of the column data type STRUCT.""") } test("UNSUPPORTED_FEATURE: unsupported pivot operations") { @@ -243,7 +243,7 @@ class QueryExecutionErrorsSuite extends QueryTest assert(e.getErrorClass === "UNSUPPORTED_OPERATION") assert(e.getMessage === "The operation is not supported: " + - "timestamp must supply timeZoneId parameter while converting to ArrowType") + "TIMESTAMP must supply timeZoneId parameter while converting to the arrow timestamp type.") } test("UNSUPPORTED_OPERATION - SPARK-36346: can't read Timestamp as TimestampNTZ") { @@ -256,7 +256,7 @@ class QueryExecutionErrorsSuite extends QueryTest assert(e.getErrorClass === "UNSUPPORTED_OPERATION") assert(e.getMessage === "The operation is not supported: " + - "Unable to convert timestamp of Orc to data type 'timestamp_ntz'") + "Unable to convert TIMESTAMP of Orc to data type TIMESTAMP_NTZ.") } } } @@ -271,7 +271,7 @@ class QueryExecutionErrorsSuite extends QueryTest assert(e.getErrorClass === "UNSUPPORTED_OPERATION") assert(e.getMessage === "The operation is not supported: " + - "Unable to convert timestamp ntz of Orc to data type 'timestamp_ltz'") + "Unable to convert TIMESTAMP_NTZ of Orc to data type TIMESTAMP.") } } } @@ -365,7 +365,7 @@ class QueryExecutionErrorsSuite extends QueryTest } assert(e.getErrorClass === "CAST_CAUSES_OVERFLOW") assert(e.getSqlState === "22005") - assert(e.getMessage === "Casting 253402258394567890L to int causes overflow. " + + assert(e.getMessage === "Casting 253402258394567890L to INT causes overflow. " + "To return NULL instead, use 'try_cast'. " + "If necessary set spark.sql.ansi.enabled to false to bypass this error.") } From 28c0d9f19c4ed7ddf2c62ac4ef21a33fdb3cbebf Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 12:18:54 +0300 Subject: [PATCH 4/8] Adjust QueryParsingErrorsSuite --- .../apache/spark/sql/errors/QueryParsingErrors.scala | 11 ++++++++--- .../spark/sql/errors/QueryParsingErrorsSuite.scala | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 69e118d88bd3b..217d998a23eba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -23,12 +23,13 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.types.StringType /** * Object for grouping all error messages of the query parsing. * Currently it includes all ParseException. */ -object QueryParsingErrors { +object QueryParsingErrors extends QueryErrorsBase { def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = { new ParseException("Invalid InsertIntoContext", ctx) @@ -303,8 +304,12 @@ object QueryParsingErrors { } def showFunctionsInvalidPatternError(pattern: String, ctx: ParserRuleContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"Invalid pattern in SHOW FUNCTIONS: $pattern. It must be a string literal."), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Array( + s"Invalid pattern in SHOW FUNCTIONS: $pattern. " + + s"It must be a ${toSQLType(StringType)} literal."), + ctx) } def duplicateCteDefinitionNamesError(duplicateNames: String, ctx: CtesContext): Throwable = { 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 fe92eab20f753..5292af8ea4572 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 @@ -251,7 +251,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { test("INVALID_SQL_SYNTAX: Invalid pattern in show functions") { val errorDesc = - "Invalid pattern in SHOW FUNCTIONS: f1. It must be a string literal.(line 1, pos 21)" + "Invalid pattern in SHOW FUNCTIONS: f1. It must be a STRING literal.(line 1, pos 21)" validateParsingError( sqlText = "SHOW FUNCTIONS IN db f1", From 1cc78a3701a47142ee5342ce4f199480dc3c5f81 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 12:22:51 +0300 Subject: [PATCH 5/8] Adjust QueryCompilationErrorsSuite --- .../apache/spark/sql/errors/QueryCompilationErrors.scala | 6 +++--- .../spark/sql/errors/QueryCompilationErrorsSuite.scala | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 479127d8a6bf7..5794edf2ddda7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -46,7 +46,7 @@ import org.apache.spark.sql.types._ * As commands are executed eagerly, this also includes errors thrown during the execution of * commands, which users can see immediately. */ -object QueryCompilationErrors { +object QueryCompilationErrors extends QueryErrorsBase { def groupingIDMismatchError(groupingID: GroupingID, groupByExprs: Seq[Expression]): Throwable = { new AnalysisException( @@ -161,8 +161,8 @@ object QueryCompilationErrors { errorClass = "CANNOT_UP_CAST_DATATYPE", messageParameters = Array( fromStr, - from.dataType.catalogString, - to.catalogString, + toSQLType(from.dataType), + toSQLType(to), s"The type path of the target object is:\n" + walkedTypePath.mkString("", "\n", "\n") + "You can either add an explicit cast to the input data or choose a higher precision " + "type of the field in the target object" 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 de671df74c808..472e7ef85575b 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 @@ -40,7 +40,7 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { }.message assert(msg1 === s""" - |Cannot up cast b from bigint to int. + |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" @@ -54,7 +54,7 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { }.message assert(msg2 === s""" - |Cannot up cast b.`b` from decimal(38,18) to bigint. + |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") From 66a37373fc2f0cdcb91655c8f1cd31c40cc4e0d2 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 19:22:53 +0300 Subject: [PATCH 6/8] Fix test suites --- .../encoders/EncoderResolutionSuite.scala | 8 +-- .../expressions/AnsiCastSuiteBase.scala | 4 +- .../sql/catalyst/expressions/CastSuite.scala | 66 +++++++++---------- 3 files changed, 39 insertions(+), 39 deletions(-) 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 6b4e994bb9979..b10da3efc7003 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 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") @@ -212,7 +212,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq(attr) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == s""" - |Cannot up cast a from ${attr.dataType.catalogString} to string. + |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 @@ -226,7 +226,7 @@ class EncoderResolutionSuite extends PlanTest { }.message assert(msg1 == s""" - |Cannot up cast b from bigint to int. + |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" @@ -239,7 +239,7 @@ class EncoderResolutionSuite extends PlanTest { }.message assert(msg2 == s""" - |Cannot up cast b.`b` from decimal(38,18) to bigint. + |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/expressions/AnsiCastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala index 6494fb29fda59..785fd95692e9b 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 @@ -279,7 +279,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { assert(negativeTs.getTime < 0) Seq(ByteType, ShortType, IntegerType).foreach { dt => checkExceptionInExpression[SparkArithmeticException]( - cast(negativeTs, dt), s"to ${dt.catalogString} causes overflow") + cast(negativeTs, dt), s"to ${dt.sql} causes overflow") } } } @@ -290,7 +290,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { assert(negativeTs.getTime < 0) Seq(ByteType, ShortType, IntegerType).foreach { dt => checkExceptionInExpression[SparkArithmeticException]( - cast(negativeTs, dt), s"to ${dt.catalogString} causes overflow") + cast(negativeTs, dt), s"to ${dt.sql} causes overflow") } val expectedSecs = Math.floorDiv(negativeTs.getTime, MILLIS_PER_SECOND) checkEvaluation(cast(negativeTs, LongType), expectedSecs) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index b6c347cfedb75..fe53dd0e5816a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -592,15 +592,15 @@ class CastSuite extends CastSuiteBase { val e1 = intercept[ArithmeticException] { Cast(Literal(Byte.MaxValue + 1), ByteType).eval() }.getMessage - assert(e1.contains("Casting 128 to tinyint causes overflow")) + assert(e1.contains("Casting 128 to TINYINT causes overflow")) val e2 = intercept[ArithmeticException] { Cast(Literal(Short.MaxValue + 1), ShortType).eval() }.getMessage - assert(e2.contains("Casting 32768 to smallint causes overflow")) + assert(e2.contains("Casting 32768 to SMALLINT causes overflow")) val e3 = intercept[ArithmeticException] { Cast(Literal(Int.MaxValue + 1L), IntegerType).eval() }.getMessage - assert(e3.contains("Casting 2147483648L to int causes overflow")) + assert(e3.contains("Casting 2147483648L to INT causes overflow")) } } @@ -642,15 +642,15 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(v2, LongType), 25L) case MINUTE => checkExceptionInExpression[ArithmeticException](cast(v2, ByteType), - s"Casting $v2 to tinyint causes overflow") + s"Casting $v2 to TINYINT causes overflow") checkEvaluation(cast(v2, ShortType), (MINUTES_PER_HOUR * 25 + 1).toShort) checkEvaluation(cast(v2, IntegerType), (MINUTES_PER_HOUR * 25 + 1).toInt) checkEvaluation(cast(v2, LongType), MINUTES_PER_HOUR * 25 + 1) case SECOND => checkExceptionInExpression[ArithmeticException](cast(v2, ByteType), - s"Casting $v2 to tinyint causes overflow") + s"Casting $v2 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v2, ShortType), - s"Casting $v2 to smallint causes overflow") + s"Casting $v2 to SMALLINT causes overflow") checkEvaluation(cast(v2, IntegerType), num.toInt) checkEvaluation(cast(v2, LongType), num) } @@ -659,34 +659,34 @@ class CastSuite extends CastSuiteBase { dt.endField match { case DAY => checkExceptionInExpression[ArithmeticException](cast(v3, ByteType), - s"Casting $v3 to tinyint causes overflow") + s"Casting $v3 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, ShortType), - s"Casting $v3 to smallint causes overflow") + s"Casting $v3 to SMALLINT causes overflow") checkEvaluation(cast(v3, IntegerType), (Long.MaxValue / MICROS_PER_DAY).toInt) checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_DAY) case HOUR => checkExceptionInExpression[ArithmeticException](cast(v3, ByteType), - s"Casting $v3 to tinyint causes overflow") + s"Casting $v3 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, ShortType), - s"Casting $v3 to smallint causes overflow") + s"Casting $v3 to SMALLINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, IntegerType), - s"Casting $v3 to int causes overflow") + s"Casting $v3 to INT causes overflow") checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_HOUR) case MINUTE => checkExceptionInExpression[ArithmeticException](cast(v3, ByteType), - s"Casting $v3 to tinyint causes overflow") + s"Casting $v3 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, ShortType), - s"Casting $v3 to smallint causes overflow") + s"Casting $v3 to SMALLINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, IntegerType), - s"Casting $v3 to int causes overflow") + s"Casting $v3 to INT causes overflow") checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_MINUTE) case SECOND => checkExceptionInExpression[ArithmeticException](cast(v3, ByteType), - s"Casting $v3 to tinyint causes overflow") + s"Casting $v3 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, ShortType), - s"Casting $v3 to smallint causes overflow") + s"Casting $v3 to SMALLINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v3, IntegerType), - s"Casting $v3 to int causes overflow") + s"Casting $v3 to INT causes overflow") checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_SECOND) } @@ -694,34 +694,34 @@ class CastSuite extends CastSuiteBase { dt.endField match { case DAY => checkExceptionInExpression[ArithmeticException](cast(v4, ByteType), - s"Casting $v4 to tinyint causes overflow") + s"Casting $v4 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, ShortType), - s"Casting $v4 to smallint causes overflow") + s"Casting $v4 to SMALLINT causes overflow") checkEvaluation(cast(v4, IntegerType), (Long.MinValue / MICROS_PER_DAY).toInt) checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_DAY) case HOUR => checkExceptionInExpression[ArithmeticException](cast(v4, ByteType), - s"Casting $v4 to tinyint causes overflow") + s"Casting $v4 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, ShortType), - s"Casting $v4 to smallint causes overflow") + s"Casting $v4 to SMALLINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, IntegerType), - s"Casting $v4 to int causes overflow") + s"Casting $v4 to INT causes overflow") checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_HOUR) case MINUTE => checkExceptionInExpression[ArithmeticException](cast(v4, ByteType), - s"Casting $v4 to tinyint causes overflow") + s"Casting $v4 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, ShortType), - s"Casting $v4 to smallint causes overflow") + s"Casting $v4 to SMALLINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, IntegerType), - s"Casting $v4 to int causes overflow") + s"Casting $v4 to INT causes overflow") checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_MINUTE) case SECOND => checkExceptionInExpression[ArithmeticException](cast(v4, ByteType), - s"Casting $v4 to tinyint causes overflow") + s"Casting $v4 to TINYINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, ShortType), - s"Casting $v4 to smallint causes overflow") + s"Casting $v4 to SMALLINT causes overflow") checkExceptionInExpression[ArithmeticException](cast(v4, IntegerType), - s"Casting $v4 to int causes overflow") + s"Casting $v4 to INT causes overflow") checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_SECOND) } } @@ -777,7 +777,7 @@ class CastSuite extends CastSuiteBase { ).foreach { case (v, toType) => checkExceptionInExpression[ArithmeticException](cast(v, toType), - s"Casting $v to ${toType.catalogString} causes overflow") + s"Casting $v to ${toType.sql} causes overflow") } Seq( @@ -792,7 +792,7 @@ class CastSuite extends CastSuiteBase { ).foreach { case (v, toType) => checkExceptionInExpression[ArithmeticException](cast(v, toType), - s"Casting ${v}L to ${toType.catalogString} causes overflow") + s"Casting ${v}L to ${toType.sql} causes overflow") } } @@ -829,7 +829,7 @@ class CastSuite extends CastSuiteBase { case (v, dt, toType) => val value = Literal.create(v, dt) checkExceptionInExpression[ArithmeticException](cast(value, toType), - s"Casting $value to ${toType.catalogString} causes overflow") + s"Casting $value to ${toType.sql} causes overflow") } Seq( @@ -887,7 +887,7 @@ class CastSuite extends CastSuiteBase { ).foreach { case (v, toType) => checkExceptionInExpression[ArithmeticException](cast(v, toType), - s"Casting $v to ${toType.catalogString} causes overflow") + s"Casting $v to ${toType.sql} causes overflow") } Seq( @@ -898,7 +898,7 @@ class CastSuite extends CastSuiteBase { ).foreach { case (v, toType) => checkExceptionInExpression[ArithmeticException](cast(v, toType), - s"Casting ${v}L to ${toType.catalogString} causes overflow") + s"Casting ${v}L to ${toType.sql} causes overflow") } } } From ab12738354307e74e0d7d822e314b240703ff509 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 19:31:39 +0300 Subject: [PATCH 7/8] Re-gen sql.out --- .../resources/sql-tests/results/postgreSQL/float4.sql.out | 6 +++--- .../resources/sql-tests/results/postgreSQL/float8.sql.out | 2 +- .../resources/sql-tests/results/postgreSQL/int8.sql.out | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) 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 39636e02159eb..a7ee7400e58bd 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 @@ -340,7 +340,7 @@ SELECT int(float('2147483647')) struct<> -- !query output org.apache.spark.SparkArithmeticException -Casting 2.14748365E9 to int causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting 2.14748365E9 to INT causes overflow. To return NULL instead, use 'try_cast'. 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 -Casting -2.1474839E9 to int causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting -2.1474839E9 to INT causes overflow. To return NULL instead, use 'try_cast'. 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 -Casting -9.22338E18 to bigint causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting -9.22338E18 to BIGINT causes overflow. To return NULL instead, use 'try_cast'. 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 b2f61306c7c7c..3237969ea8736 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 @@ -845,7 +845,7 @@ SELECT bigint(double('-9223372036854780000')) struct<> -- !query output org.apache.spark.SparkArithmeticException -Casting -9.22337203685478E18D to bigint causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting -9.22337203685478E18D to BIGINT causes overflow. To return NULL instead, use 'try_cast'. 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/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index b7185fcbf1fea..be1fce4b41e7c 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 @@ -619,7 +619,7 @@ SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 struct<> -- !query output org.apache.spark.SparkArithmeticException -Casting 4567890123456789L to int causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting 4567890123456789L to INT causes overflow. To return NULL instead, use 'try_cast'. 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 -Casting 4567890123456789L to smallint causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting 4567890123456789L to SMALLINT causes overflow. To return NULL instead, use 'try_cast'. 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 -Casting 9.223372036854776E20D to bigint causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting 9.223372036854776E20D to BIGINT causes overflow. To return NULL instead, use 'try_cast'. 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 -Casting -9223372036854775808L to int causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. +Casting -9223372036854775808L to INT causes overflow. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. -- !query From e1ad0c05e967ca8ee36c25f4add44f5ef04f59b4 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sun, 17 Apr 2022 19:44:27 +0300 Subject: [PATCH 8/8] Fix tests --- .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/InsertSuite.scala | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index c846441e9e009..9133205e82028 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1951,7 +1951,7 @@ class DatasetSuite extends QueryTest .map(b => b - 1) .collect() } - assert(thrownException.message.contains("Cannot up cast id from bigint to tinyint")) + assert(thrownException.message.contains("Cannot up cast id from BIGINT to TINYINT")) } test("SPARK-26690: checkpoints should be executed with an execution id") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 8d921d47e83db..bf07f643cf126 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -721,13 +721,13 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { var msg = intercept[SparkException] { sql(s"insert into t values($outOfRangeValue1)") }.getCause.getMessage - assert(msg.contains(s"Casting ${outOfRangeValue1}L to int causes overflow")) + assert(msg.contains(s"Casting ${outOfRangeValue1}L to INT causes overflow")) val outOfRangeValue2 = (Int.MinValue - 1L).toString msg = intercept[SparkException] { sql(s"insert into t values($outOfRangeValue2)") }.getCause.getMessage - assert(msg.contains(s"Casting ${outOfRangeValue2}L to int causes overflow")) + assert(msg.contains(s"Casting ${outOfRangeValue2}L to INT causes overflow")) } } } @@ -741,13 +741,13 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { var msg = intercept[SparkException] { sql(s"insert into t values(${outOfRangeValue1}D)") }.getCause.getMessage - assert(msg.contains(s"Casting ${outOfRangeValue1}D to bigint causes overflow")) + assert(msg.contains(s"Casting ${outOfRangeValue1}D to BIGINT causes overflow")) val outOfRangeValue2 = Math.nextDown(Long.MinValue) msg = intercept[SparkException] { sql(s"insert into t values(${outOfRangeValue2}D)") }.getCause.getMessage - assert(msg.contains(s"Casting ${outOfRangeValue2}D to bigint causes overflow")) + assert(msg.contains(s"Casting ${outOfRangeValue2}D to BIGINT causes overflow")) } } }