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
56 changes: 56 additions & 0 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,31 @@
"message" : [ "Field name %s is ambiguous and has %s matching fields in the struct." ],
"sqlState" : "42000"
},
"CANNOT_CAST_DATATYPE" : {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any ideas how to trigger this error class? I haven't found any ways how trigger it from user code. WDYT? @cloud-fan @HyukjinKwon

"message" : [ "Cannot cast %s to %s." ],
"sqlState" : "22005"
},
"CANNOT_CHANGE_DECIMAL_PRECISION" : {
"message" : [ "%s cannot be represented as Decimal(%s, %s)." ],
"sqlState" : "22005"
},
"CANNOT_EVALUATE_EXPRESSION" : {
"message" : [ "Cannot evaluate expression: %s" ]
},
"CANNOT_GENERATE_CODE_FOR_EXPRESSION" : {
"message" : [ "Cannot generate code for expression: %s" ]
},
"CANNOT_PARSE_DECIMAL" : {
"message" : [ "Cannot parse decimal" ],
"sqlState" : "42000"
},
"CANNOT_TERMINATE_GENERATOR" : {
"message" : [ "Cannot terminate expression: %s" ]
},
"CAST_CAUSES_OVERFLOW" : {
"message" : [ "Casting %s to %s causes overflow" ],
"sqlState" : "22005"
},
"CONCURRENT_QUERY" : {
"message" : [ "Another instance of this query was just started by a concurrent session." ]
},
Expand All @@ -14,6 +39,9 @@
"message" : [ "Found duplicate keys '%s'" ],
"sqlState" : "23000"
},
"FAILED_EXECUTE_UDF" : {
"message" : [ "Failed to execute user defined function (%s: (%s) => %s)" ]
},
"FAILED_RENAME_PATH" : {
"message" : [ "Failed to rename %s to %s as destination already exists" ],
"sqlState" : "22023"
Expand Down Expand Up @@ -46,6 +74,9 @@
"message" : [ "Index %s must be between 0 and the length of the ArrayData." ],
"sqlState" : "22023"
},
"INVALID_ARRAY_INDEX" : {
"message" : [ "Invalid index: %s, numElements: %s" ]
},
"INVALID_FIELD_NAME" : {
"message" : [ "Field name %s is invalid: %s is not a struct." ],
"sqlState" : "42000"
Expand All @@ -54,9 +85,19 @@
"message" : [ "The fraction of sec must be zero. Valid range is [0, 60]." ],
"sqlState" : "22023"
},
"INVALID_INPUT_SYNTAX_FOR_NUMERIC_TYPE" : {
"message" : [ "invalid input syntax for type numeric: %s" ],
"sqlState" : "42000"
},
"INVALID_JSON_SCHEMA_MAPTYPE" : {
"message" : [ "Input schema %s can only contain StringType as a key type for a MapType." ]
},
"LOGICAL_HINT_OPERATOR_NOT_REMOVED_DURING_ANALYSIS" : {
"message" : [ "Internal error: logical hint operator should have been removed during analysis" ]
},
"MAP_KEY_DOES_NOT_EXIST" : {
"message" : [ "Key %s does not exist." ]
},
"MISSING_COLUMN" : {
"message" : [ "cannot resolve '%s' given input columns: [%s]" ],
"sqlState" : "42000"
Expand Down Expand Up @@ -85,6 +126,10 @@
"message" : [ "Failed to rename as %s was not found" ],
"sqlState" : "22023"
},
"ROW_FROM_CSV_PARSER_NOT_EXPECTED" : {
"message" : [ "Expected one row from CSV parser." ],
"sqlState" : "42000"
},
"SECOND_FUNCTION_ARGUMENT_NOT_INTEGER" : {
"message" : [ "The second argument of '%s' function needs to be an integer." ],
"sqlState" : "22023"
Expand All @@ -96,10 +141,21 @@
"message" : [ "Unrecognized SQL type %s" ],
"sqlState" : "42000"
},
"UNSUPPORTED_CHANGE_COLUMN" : {
"message" : [ "Please add an implementation for a column change here" ],
"sqlState" : "0A000"
},
"UNSUPPORTED_DATATYPE" : {
"message" : [ "Unsupported data type %s" ],
"sqlState" : "0A000"
},
"UNSUPPORTED_LITERAL_TYPE" : {
"message" : [ "Unsupported literal type %s %s" ],
"sqlState" : "0A000"
},
"UNSUPPORTED_SIMPLE_STRING_WITH_NODE_ID" : {
"message" : [ "%s does not implement simpleStringWithNodeId" ]
},
"UNSUPPORTED_TRANSACTION_BY_JDBC_SERVER" : {
"message" : [ "The target JDBC server does not support transaction and can only support ALTER TABLE with a single action." ],
"sqlState" : "0A000"
Expand Down
78 changes: 78 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkException.scala
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,19 @@ private[spark] class SparkArithmeticException(errorClass: String, messageParamet
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Unsupported operation exception thrown from Spark with an error class.
*/
private[spark] class SparkUnsupportedOperationException(
errorClass: String,
messageParameters: Array[String])
extends UnsupportedOperationException(
SparkThrowableHelper.getMessage(errorClass, messageParameters)) with SparkThrowable {

override def getErrorClass: String = errorClass
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Class not found exception thrown from Spark with an error class.
*/
Expand Down Expand Up @@ -139,6 +152,19 @@ private[spark] class SparkFileAlreadyExistsException(
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Illegal state exception thrown from Spark with an error class.
*/
private[spark] class SparkIllegalStateException(
errorClass: String,
messageParameters: Array[String])
extends IllegalStateException(
SparkThrowableHelper.getMessage(errorClass, messageParameters)) with SparkThrowable {

override def getErrorClass: String = errorClass
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* File not found exception thrown from Spark with an error class.
*/
Expand All @@ -152,6 +178,19 @@ private[spark] class SparkFileNotFoundException(
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Number format exception thrown from Spark with an error class.
*/
private[spark] class SparkNumberFormatException(
errorClass: String,
messageParameters: Array[String])
extends NumberFormatException(
SparkThrowableHelper.getMessage(errorClass, messageParameters)) with SparkThrowable {

override def getErrorClass: String = errorClass
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* No such method exception thrown from Spark with an error class.
*/
Expand All @@ -165,6 +204,19 @@ private[spark] class SparkNoSuchMethodException(
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Illegal argument exception thrown from Spark with an error class.
*/
private[spark] class SparkIllegalArgumentException(
errorClass: String,
messageParameters: Array[String])
extends IllegalArgumentException(
SparkThrowableHelper.getMessage(errorClass, messageParameters)) with SparkThrowable {

override def getErrorClass: String = errorClass
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Index out of bounds exception thrown from Spark with an error class.
*/
Expand Down Expand Up @@ -215,6 +267,19 @@ private[spark] class SparkSecurityException(
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* Array index out of bounds exception thrown from Spark with an error class.
*/
private[spark] class SparkArrayIndexOutOfBoundsException(
errorClass: String,
messageParameters: Array[String])
extends ArrayIndexOutOfBoundsException(
SparkThrowableHelper.getMessage(errorClass, messageParameters)) with SparkThrowable {

override def getErrorClass: String = errorClass
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* SQL exception thrown from Spark with an error class.
*/
Expand All @@ -228,6 +293,19 @@ private[spark] class SparkSQLException(
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* No such element exception thrown from Spark with an error class.
*/
private[spark] class SparkNoSuchElementException(
errorClass: String,
messageParameters: Array[String])
extends NoSuchElementException(
SparkThrowableHelper.getMessage(errorClass, messageParameters)) with SparkThrowable {

override def getErrorClass: String = errorClass
override def getSqlState: String = SparkThrowableHelper.getSqlState(errorClass)
}

/**
* SQL feature not supported exception thrown from Spark with an error class.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.permission.FsPermission
import org.codehaus.commons.compiler.CompileException
import org.codehaus.janino.InternalCompilerException

import org.apache.spark.{Partition, SparkArithmeticException, SparkClassNotFoundException, SparkConcurrentModificationException, SparkDateTimeException, SparkException, SparkFileAlreadyExistsException, SparkFileNotFoundException, SparkIndexOutOfBoundsException, SparkNoSuchMethodException, SparkRuntimeException, SparkSecurityException, SparkSQLException, SparkSQLFeatureNotSupportedException, SparkUpgradeException}
import org.apache.spark.{Partition, SparkArithmeticException, SparkArrayIndexOutOfBoundsException, SparkClassNotFoundException, SparkConcurrentModificationException, SparkDateTimeException, SparkException, SparkFileAlreadyExistsException, SparkFileNotFoundException, SparkIllegalArgumentException, SparkIllegalStateException, SparkIndexOutOfBoundsException, SparkNoSuchElementException, SparkNoSuchMethodException, SparkNumberFormatException, SparkRuntimeException, SparkSecurityException, SparkSQLException, SparkSQLFeatureNotSupportedException, SparkUnsupportedOperationException, SparkUpgradeException}
import org.apache.spark.executor.CommitDeniedException
import org.apache.spark.launcher.SparkLauncher
import org.apache.spark.memory.SparkOutOfMemoryError
Expand Down Expand Up @@ -67,89 +67,106 @@ import org.apache.spark.util.CircularBuffer
object QueryExecutionErrors {

def columnChangeUnsupportedError(): Throwable = {
new UnsupportedOperationException("Please add an implementation for a column change here")
new SparkUnsupportedOperationException(errorClass = "UNSUPPORTED_CHANGE_COLUMN",
messageParameters = Array.empty)
}

def logicalHintOperatorNotRemovedDuringAnalysisError(): Throwable = {
new IllegalStateException(
"Internal error: logical hint operator should have been removed during analysis")
new SparkIllegalStateException(errorClass = "LOGICAL_HINT_OPERATOR_NOT_REMOVED_DURING_ANALYSIS",
messageParameters = Array.empty)
}

def cannotEvaluateExpressionError(expression: Expression): Throwable = {
new UnsupportedOperationException(s"Cannot evaluate expression: $expression")
new SparkUnsupportedOperationException(errorClass = "CANNOT_EVALUATE_EXPRESSION",
messageParameters = Array("", expression.toString))
}

def cannotGenerateCodeForExpressionError(expression: Expression): Throwable = {
new UnsupportedOperationException(s"Cannot generate code for expression: $expression")
new SparkUnsupportedOperationException(errorClass = "CANNOT_GENERATE_CODE_FOR_EXPRESSION",
messageParameters = Array(expression.toString))
}

def cannotTerminateGeneratorError(generator: UnresolvedGenerator): Throwable = {
new UnsupportedOperationException(s"Cannot terminate expression: $generator")
new SparkUnsupportedOperationException(errorClass = "CANNOT_TERMINATE_GENERATOR",
messageParameters = Array(generator.toString))
}

def castingCauseOverflowError(t: Any, targetType: String): ArithmeticException = {
new ArithmeticException(s"Casting $t to $targetType causes overflow")
new SparkArithmeticException (errorClass = "CAST_CAUSES_OVERFLOW",
messageParameters = Array(t.toString, targetType))
}

def cannotChangeDecimalPrecisionError(
value: Decimal, decimalPrecision: Int, decimalScale: Int): ArithmeticException = {
new ArithmeticException(s"${value.toDebugString} cannot be represented as " +
s"Decimal($decimalPrecision, $decimalScale).")
new SparkArithmeticException(errorClass = "CANNOT_CHANGE_DECIMAL_PRECISION",
messageParameters = Array(value.toDebugString,
decimalPrecision.toString, decimalScale.toString))
}

def invalidInputSyntaxForNumericError(s: UTF8String): NumberFormatException = {
new NumberFormatException(s"invalid input syntax for type numeric: $s")
new SparkNumberFormatException(errorClass = "INVALID_INPUT_SYNTAX_FOR_NUMERIC_TYPE",
messageParameters = Array(s.toString))
}

def cannotCastFromNullTypeError(to: DataType): Throwable = {
new SparkException(s"should not directly cast from NullType to $to.")
new SparkException(errorClass = "CANNOT_CAST_DATATYPE",
messageParameters = Array(NullType.typeName, to.typeName), null)
}

def cannotCastError(from: DataType, to: DataType): Throwable = {
new SparkException(s"Cannot cast $from to $to.")
new SparkException(errorClass = "CANNOT_CAST_DATATYPE",
messageParameters = Array(from.typeName, to.typeName), null)
}

def cannotParseDecimalError(): Throwable = {
new IllegalArgumentException("Cannot parse any decimal")
new SparkIllegalStateException(errorClass = "CANNOT_PARSE_DECIMAL",
messageParameters = Array.empty)
}

def simpleStringWithNodeIdUnsupportedError(nodeName: String): Throwable = {
new UnsupportedOperationException(s"$nodeName does not implement simpleStringWithNodeId")
new SparkUnsupportedOperationException(errorClass = "UNSUPPORTED_SIMPLE_STRING_WITH_NODE_ID",
messageParameters = Array(nodeName))
}

def evaluateUnevaluableAggregateUnsupportedError(
methodName: String, unEvaluable: UnevaluableAggregate): Throwable = {
new UnsupportedOperationException(s"Cannot evaluate $methodName: $unEvaluable")
new SparkUnsupportedOperationException(errorClass = "CANNOT_EVALUATE_EXPRESSION",
messageParameters = Array(methodName + ": " + unEvaluable.toString))
}

def dataTypeUnsupportedError(dt: DataType): Throwable = {
new SparkException(s"Unsupported data type $dt")
new SparkException(errorClass = "UNSUPPORTED_DATATYPE",
messageParameters = Array(dt.typeName), null)
}

def dataTypeUnsupportedError(dataType: String, failure: String): Throwable = {
new IllegalArgumentException(s"Unsupported dataType: $dataType, $failure")
new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_DATATYPE",
messageParameters = Array(dataType + failure))
}

def failedExecuteUserDefinedFunctionError(funcCls: String, inputTypes: String,
outputType: String, e: Throwable): Throwable = {
new SparkException(
s"Failed to execute user defined function ($funcCls: ($inputTypes) => $outputType)", e)
new SparkException(errorClass = "FAILED_EXECUTE_UDF",
messageParameters = Array(funcCls, inputTypes, outputType), e)
}

def divideByZeroError(): ArithmeticException = {
new SparkArithmeticException(errorClass = "DIVIDE_BY_ZERO", messageParameters = Array.empty)
}

def invalidArrayIndexError(index: Int, numElements: Int): ArrayIndexOutOfBoundsException = {
new ArrayIndexOutOfBoundsException(s"Invalid index: $index, numElements: $numElements")
new SparkArrayIndexOutOfBoundsException(errorClass = "INVALID_ARRAY_INDEX",
messageParameters = Array(index.toString, numElements.toString))
}

def mapKeyNotExistError(key: Any): NoSuchElementException = {
new NoSuchElementException(s"Key $key does not exist.")
new SparkNoSuchElementException(errorClass = "MAP_KEY_DOES_NOT_EXIST",
messageParameters = Array(key.toString))
}

def rowFromCSVParserNotExpectedError(): Throwable = {
new IllegalArgumentException("Expected one row from CSV parser.")
new SparkIllegalArgumentException(errorClass = "ROW_FROM_CSV_PARSER_NOT_EXPECTED",
messageParameters = Array.empty)
}

def inputTypeUnsupportedError(dataType: DataType): Throwable = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import java.util.Arrays
import scala.collection.mutable.ArrayBuffer
import scala.reflect.runtime.universe.TypeTag

import org.apache.spark.SparkArithmeticException
import org.apache.spark.sql.{Encoder, Encoders}
import org.apache.spark.sql.catalyst.{FooClassWithEnum, FooEnum, OptionalData, PrimitiveData}
import org.apache.spark.sql.catalyst.analysis.AnalysisTest
Expand Down Expand Up @@ -614,7 +615,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes
toRow(bigNumeric)
}
assert(e.getMessage.contains("Error while encoding"))
assert(e.getCause.getClass === classOf[ArithmeticException])
assert(e.getCause.getClass === classOf[SparkArithmeticException])
}
}
}
Expand Down
Loading