Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,4 +53,8 @@ trait QueryErrorsBase {
def toSQLId(parts: String): String = {
toSQLId(parts.split("\\."))
}

def toSQLType(t: DataType): String = {
t.sql
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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 = {
Expand Down Expand Up @@ -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.")
)
}

Expand Down Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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 = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down Expand Up @@ -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
Expand All @@ -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"
Expand All @@ -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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
}
}
Expand All @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
}
}

Expand Down Expand Up @@ -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)
}
Expand All @@ -659,69 +659,69 @@ 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)
}

val v4 = Literal.create(Duration.of(Long.MinValue, ChronoUnit.MICROS), dt)
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)
}
}
Expand Down Expand Up @@ -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(
Expand All @@ -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")
}
}

Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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(
Expand All @@ -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")
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
Loading