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 6b32a08b6fd75..0710875558645 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 @@ -162,8 +162,8 @@ object QueryCompilationErrors extends QueryErrorsBase { 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/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 9b18b59c33dde..b7989d119863d 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 @@ -53,4 +53,8 @@ trait QueryErrorsBase { def toSQLId(parts: String): String = { toSQLId(parts.split("\\.")) } + + def toSQLType(t: DataType): String = { + t.sql + } } 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 1aef33c6cc20c..79e36cb485a97 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.") ) } @@ -1928,14 +1927,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/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 ba9c9ecbadecc..b3785b2062365 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 @@ -22,6 +22,7 @@ import org.antlr.v4.runtime.ParserRuleContext 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.types.StringType /** * Object for grouping all error messages of the query parsing. @@ -305,9 +306,12 @@ object QueryParsingErrors extends QueryErrorsBase { } def showFunctionsInvalidPatternError(pattern: String, ctx: ParserRuleContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"Invalid pattern in SHOW FUNCTIONS: ${toSQLId(pattern)}. " + - "It must be a string literal."), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Array( + s"Invalid pattern in SHOW FUNCTIONS: ${toSQLId(pattern)}. " + + s"It must be a ${toSQLType(StringType)} literal."), + ctx) } def duplicateCteDefinitionNamesError(duplicateNames: String, ctx: CtesContext): 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 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") } } } 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 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/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 9eb8f98ed55ed..0926cacbffcca 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") 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 8f79721090491..85956bd88768b 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.") } 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 29ed18071a4c2..bf8aa9494eb6a 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 @@ -257,7 +257,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { sqlState = "42000", message = s""" - |Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a string literal.(line 1, pos 21) + |Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a STRING literal.(line 1, pos 21) | |== SQL == |SHOW FUNCTIONS IN db f1 @@ -269,7 +269,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { sqlState = "42000", message = s""" - |Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a string literal.(line 1, pos 26) + |Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a STRING literal.(line 1, pos 26) | |== SQL == |SHOW FUNCTIONS IN db LIKE f1 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")) } } }