This is an automated email from the ASF dual-hosted git repository.
maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 42d4f848054 [SPARK-40663][SQL] Migrate execution errors onto error
classes: _LEGACY_ERROR_TEMP_2151-2175
42d4f848054 is described below
commit 42d4f848054554c251448009256ff9ecd80da920
Author: itholic <[email protected]>
AuthorDate: Thu Oct 13 08:47:33 2022 +0300
[SPARK-40663][SQL] Migrate execution errors onto error classes:
_LEGACY_ERROR_TEMP_2151-2175
### What changes were proposed in this pull request?
This PR proposes to migrate 25 execution errors onto temporary error
classes with the prefix `_LEGACY_ERROR_TEMP_2151` to `_LEGACY_ERROR_TEMP_2175`.
The error classes are prefixed with `_LEGACY_ERROR_TEMP_` indicates the
dev-facing error messages, and won't be exposed to end users.
### Why are the changes needed?
To speed-up the error class migration.
The migration on temporary error classes allow us to analyze the errors, so
we can detect the most popular error classes.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
```
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite"
$ build/sbt "test:testOnly *SQLQuerySuite"
$ build/sbt -Phive-thriftserver "hive-thriftserver/testOnly
org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite"
```
Closes #38149 from itholic/SPARK-40540-2151-2200.
Authored-by: itholic <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
---
core/src/main/resources/error/error-classes.json | 131 +++++++++++
.../spark/sql/errors/QueryExecutionErrors.scala | 242 +++++++++++++--------
2 files changed, 282 insertions(+), 91 deletions(-)
diff --git a/core/src/main/resources/error/error-classes.json
b/core/src/main/resources/error/error-classes.json
index dd95c0f83d1..2834dee231a 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -3805,5 +3805,136 @@
"message" : [
"Due to Scala's limited support of tuple, tuple with more than 22
elements are not supported."
]
+ },
+ "_LEGACY_ERROR_TEMP_2151" : {
+ "message" : [
+ "Error while decoding: <e>",
+ "<expressions>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2152" : {
+ "message" : [
+ "Error while encoding: <e>",
+ "<expressions>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2153" : {
+ "message" : [
+ "class <clsName> has unexpected serializer: <objSerializer>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2154" : {
+ "message" : [
+ "Failed to get outer pointer for <innerCls>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2155" : {
+ "message" : [
+ "<userClass> is not annotated with SQLUserDefinedType nor registered
with UDTRegistration.}"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2156" : {
+ "message" : [
+ "The size function doesn't support the operand type <dataType>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2157" : {
+ "message" : [
+ "Unexpected value for start in function <prettyName>: SQL array indices
start at 1."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2158" : {
+ "message" : [
+ "Unexpected value for length in function <prettyName>: length must be
greater than or equal to 0."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2159" : {
+ "message" : [
+ "Unsuccessful try to concat arrays with <numberOfElements> elements due
to exceeding the array size limit <maxRoundedArrayLength>."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2160" : {
+ "message" : [
+ "Unsuccessful try to flatten an array of arrays with <numberOfElements>
elements due to exceeding the array size limit <maxRoundedArrayLength>."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2161" : {
+ "message" : [
+ "Unsuccessful try to create array with <count> elements due to exceeding
the array size limit <maxRoundedArrayLength>."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2162" : {
+ "message" : [
+ "Unsuccessful try to union arrays with <length> elements due to
exceeding the array size limit <maxRoundedArrayLength>."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2163" : {
+ "message" : [
+ "Initial type <dataType> must be a <target>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2164" : {
+ "message" : [
+ "Initial type <dataType> must be an <arrayType>, a <structType> or a
<mapType>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2165" : {
+ "message" : [
+ "Malformed records are detected in schema inference. Parse Mode:
<failFastMode>."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2166" : {
+ "message" : [
+ "Malformed JSON"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2167" : {
+ "message" : [
+ "Malformed records are detected in schema inference. Parse Mode:
<failFastMode>. Reasons: Failed to infer a common schema. Struct types are
expected, but `<dataType>` was found."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2168" : {
+ "message" : [
+ "Decorrelate inner query through <plan> is not supported."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2169" : {
+ "message" : [
+ "This method should not be called in the analyzer"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2170" : {
+ "message" : [
+ "Cannot safely merge SERDEPROPERTIES:",
+ "<props1>",
+ "<props2>",
+ "The conflict keys: <conflictKeys>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2171" : {
+ "message" : [
+ "Not supported pair: <r1>, <r2> at <function>()"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2172" : {
+ "message" : [
+ "Once strategy's idempotence is broken for batch <batchName>",
+ "<plan>"
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2173" : {
+ "message" : [
+ "The structural integrity of the input plan is broken in <className>."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2174" : {
+ "message" : [
+ "After applying rule <ruleName> in batch <batchName>, the structural
integrity of the plan is broken."
+ ]
+ },
+ "_LEGACY_ERROR_TEMP_2175" : {
+ "message" : [
+ "Rule id not found for <ruleName>"
+ ]
}
}
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 392b9bf6c72..c8dcc14096b 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
@@ -1554,42 +1554,70 @@ private[sql] object QueryExecutionErrors extends
QueryErrorsBase {
messageParameters = Map.empty)
}
- def expressionDecodingError(e: Exception, expressions: Seq[Expression]):
Throwable = {
- new RuntimeException(s"Error while decoding: $e\n" +
-
s"${expressions.map(_.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")}",
e)
+ def expressionDecodingError(e: Exception, expressions: Seq[Expression]):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2151",
+ messageParameters = Map(
+ "e" -> e.toString(),
+ "expressions" -> expressions.map(
+ _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")),
+ cause = e)
}
- def expressionEncodingError(e: Exception, expressions: Seq[Expression]):
Throwable = {
- new RuntimeException(s"Error while encoding: $e\n" +
-
s"${expressions.map(_.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")}",
e)
+ def expressionEncodingError(e: Exception, expressions: Seq[Expression]):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2152",
+ messageParameters = Map(
+ "e" -> e.toString(),
+ "expressions" -> expressions.map(
+ _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")),
+ cause = e)
}
- def classHasUnexpectedSerializerError(clsName: String, objSerializer:
Expression): Throwable = {
- new RuntimeException(s"class $clsName has unexpected serializer:
$objSerializer")
+ def classHasUnexpectedSerializerError(
+ clsName: String, objSerializer: Expression): SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2153",
+ messageParameters = Map(
+ "clsName" -> clsName,
+ "objSerializer" -> objSerializer.toString()))
}
- def cannotGetOuterPointerForInnerClassError(innerCls: Class[_]): Throwable =
{
- new RuntimeException(s"Failed to get outer pointer for
${innerCls.getName}")
+ def cannotGetOuterPointerForInnerClassError(innerCls: Class[_]):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2154",
+ messageParameters = Map(
+ "innerCls" -> innerCls.getName))
}
def userDefinedTypeNotAnnotatedAndRegisteredError(udt: UserDefinedType[_]):
Throwable = {
- new SparkException(s"${udt.userClass.getName} is not annotated with " +
- "SQLUserDefinedType nor registered with UDTRegistration.}")
+ new SparkException(
+ errorClass = "_LEGACY_ERROR_TEMP_2155",
+ messageParameters = Map(
+ "userClass" -> udt.userClass.getName),
+ cause = null)
}
- def unsupportedOperandTypeForSizeFunctionError(dataType: DataType):
Throwable = {
- new UnsupportedOperationException(
- s"The size function doesn't support the operand type
${dataType.getClass.getCanonicalName}")
+ def unsupportedOperandTypeForSizeFunctionError(
+ dataType: DataType): SparkUnsupportedOperationException = {
+ new SparkUnsupportedOperationException(
+ errorClass = "_LEGACY_ERROR_TEMP_2156",
+ messageParameters = Map(
+ "dataType" -> dataType.getClass.getCanonicalName))
}
- def unexpectedValueForStartInFunctionError(prettyName: String):
RuntimeException = {
- new RuntimeException(
- s"Unexpected value for start in function $prettyName: SQL array indices
start at 1.")
+ def unexpectedValueForStartInFunctionError(prettyName: String):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2157",
+ messageParameters = Map(
+ "prettyName" -> prettyName))
}
- def unexpectedValueForLengthInFunctionError(prettyName: String):
RuntimeException = {
- new RuntimeException(s"Unexpected value for length in function
$prettyName: " +
- "length must be greater than or equal to 0.")
+ def unexpectedValueForLengthInFunctionError(prettyName: String):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2158",
+ messageParameters = Map(
+ "prettyName" -> prettyName))
}
def elementAtByIndexZeroError(context: SQLQueryContext): RuntimeException = {
@@ -1601,119 +1629,151 @@ private[sql] object QueryExecutionErrors extends
QueryErrorsBase {
summary = getSummary(context))
}
- def concatArraysWithElementsExceedLimitError(numberOfElements: Long):
Throwable = {
- new RuntimeException(
- s"""
- |Unsuccessful try to concat arrays with $numberOfElements
- |elements due to exceeding the array size limit
- |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.
- """.stripMargin.replaceAll("\n", " "))
+ def concatArraysWithElementsExceedLimitError(numberOfElements: Long):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2159",
+ messageParameters = Map(
+ "numberOfElements" -> numberOfElements.toString(),
+ "maxRoundedArrayLength" ->
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
}
- def flattenArraysWithElementsExceedLimitError(numberOfElements: Long):
Throwable = {
- new RuntimeException(
- s"""
- |Unsuccessful try to flatten an array of arrays with $numberOfElements
- |elements due to exceeding the array size limit
- |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.
- """.stripMargin.replaceAll("\n", " "))
+ def flattenArraysWithElementsExceedLimitError(numberOfElements: Long):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2160",
+ messageParameters = Map(
+ "numberOfElements" -> numberOfElements.toString(),
+ "maxRoundedArrayLength" ->
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
}
- def createArrayWithElementsExceedLimitError(count: Any): RuntimeException = {
- new RuntimeException(
- s"""
- |Unsuccessful try to create array with $count elements
- |due to exceeding the array size limit
- |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.
- """.stripMargin.replaceAll("\n", " "))
+ def createArrayWithElementsExceedLimitError(count: Any):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2161",
+ messageParameters = Map(
+ "count" -> count.toString(),
+ "maxRoundedArrayLength" ->
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
}
- def unionArrayWithElementsExceedLimitError(length: Int): Throwable = {
- new RuntimeException(
- s"""
- |Unsuccessful try to union arrays with $length
- |elements due to exceeding the array size limit
- |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.
- """.stripMargin.replaceAll("\n", " "))
+ def unionArrayWithElementsExceedLimitError(length: Int):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2162",
+ messageParameters = Map(
+ "length" -> length.toString(),
+ "maxRoundedArrayLength" ->
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
}
- def initialTypeNotTargetDataTypeError(dataType: DataType, target: String):
Throwable = {
- new UnsupportedOperationException(s"Initial type ${dataType.catalogString}
must be a $target")
+ def initialTypeNotTargetDataTypeError(
+ dataType: DataType, target: String): SparkUnsupportedOperationException
= {
+ new SparkUnsupportedOperationException(
+ errorClass = "_LEGACY_ERROR_TEMP_2163",
+ messageParameters = Map(
+ "dataType" -> dataType.catalogString,
+ "target" -> target))
}
- def initialTypeNotTargetDataTypesError(dataType: DataType): Throwable = {
- new UnsupportedOperationException(
- s"Initial type ${dataType.catalogString} must be " +
- s"an ${ArrayType.simpleString}, a ${StructType.simpleString} or a
${MapType.simpleString}")
+ def initialTypeNotTargetDataTypesError(dataType: DataType):
SparkUnsupportedOperationException = {
+ new SparkUnsupportedOperationException(
+ errorClass = "_LEGACY_ERROR_TEMP_2164",
+ messageParameters = Map(
+ "dataType" -> dataType.catalogString,
+ "arrayType" -> ArrayType.simpleString,
+ "structType" -> StructType.simpleString,
+ "mapType" -> MapType.simpleString))
}
def malformedRecordsDetectedInSchemaInferenceError(e: Throwable): Throwable
= {
- new SparkException("Malformed records are detected in schema inference. " +
- s"Parse Mode: ${FailFastMode.name}.", e)
+ new SparkException(
+ errorClass = "_LEGACY_ERROR_TEMP_2165",
+ messageParameters = Map(
+ "failFastMode" -> FailFastMode.name),
+ cause = e)
}
def malformedJSONError(): Throwable = {
- new SparkException("Malformed JSON")
+ new SparkException(
+ errorClass = "_LEGACY_ERROR_TEMP_2166",
+ messageParameters = Map.empty,
+ cause = null)
}
def malformedRecordsDetectedInSchemaInferenceError(dataType: DataType):
Throwable = {
new SparkException(
- s"""
- |Malformed records are detected in schema inference.
- |Parse Mode: ${FailFastMode.name}. Reasons: Failed to infer a common
schema.
- |Struct types are expected, but `${dataType.catalogString}` was found.
- """.stripMargin.replaceAll("\n", " "))
+ errorClass = "_LEGACY_ERROR_TEMP_2167",
+ messageParameters = Map(
+ "failFastMode" -> FailFastMode.name,
+ "dataType" -> dataType.catalogString),
+ cause = null)
}
- def decorrelateInnerQueryThroughPlanUnsupportedError(plan: LogicalPlan):
Throwable = {
- new UnsupportedOperationException(
- s"Decorrelate inner query through ${plan.nodeName} is not supported.")
+ def decorrelateInnerQueryThroughPlanUnsupportedError(
+ plan: LogicalPlan): SparkUnsupportedOperationException = {
+ new SparkUnsupportedOperationException(
+ errorClass = "_LEGACY_ERROR_TEMP_2168",
+ messageParameters = Map(
+ "plan" -> plan.nodeName))
}
- def methodCalledInAnalyzerNotAllowedError(): Throwable = {
- new RuntimeException("This method should not be called in the analyzer")
+ def methodCalledInAnalyzerNotAllowedError(): SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2169",
+ messageParameters = Map.empty)
}
def cannotSafelyMergeSerdePropertiesError(
props1: Map[String, String],
props2: Map[String, String],
- conflictKeys: Set[String]): Throwable = {
- new UnsupportedOperationException(
- s"""
- |Cannot safely merge SERDEPROPERTIES:
- |${props1.map { case (k, v) => s"$k=$v" }.mkString("{", ",", "}")}
- |${props2.map { case (k, v) => s"$k=$v" }.mkString("{", ",", "}")}
- |The conflict keys: ${conflictKeys.mkString(", ")}
- |""".stripMargin)
+ conflictKeys: Set[String]): SparkUnsupportedOperationException = {
+ new SparkUnsupportedOperationException(
+ errorClass = "_LEGACY_ERROR_TEMP_2170",
+ messageParameters = Map(
+ "props1" -> props1.map { case (k, v) => s"$k=$v" }.mkString("{", ",",
"}"),
+ "props2" -> props2.map { case (k, v) => s"$k=$v" }.mkString("{", ",",
"}"),
+ "conflictKeys" -> conflictKeys.mkString(", ")))
}
def pairUnsupportedAtFunctionError(
- r1: ValueInterval, r2: ValueInterval, function: String): Throwable = {
- new UnsupportedOperationException(s"Not supported pair: $r1, $r2 at
$function()")
+ r1: ValueInterval,
+ r2: ValueInterval,
+ function: String): SparkUnsupportedOperationException = {
+ new SparkUnsupportedOperationException(
+ errorClass = "_LEGACY_ERROR_TEMP_2171",
+ messageParameters = Map(
+ "r1" -> r1.toString(),
+ "r2" -> r2.toString(),
+ "function" -> function))
}
def onceStrategyIdempotenceIsBrokenForBatchError[TreeType <: TreeNode[_]](
- batchName: String, plan: TreeType, reOptimized: TreeType): Throwable = {
- new RuntimeException(
- s"""
- |Once strategy's idempotence is broken for batch $batchName
- |${sideBySide(plan.treeString, reOptimized.treeString).mkString("\n")}
- """.stripMargin)
+ batchName: String, plan: TreeType, reOptimized: TreeType):
SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2172",
+ messageParameters = Map(
+ "batchName" -> batchName,
+ "plan" -> sideBySide(plan.treeString,
reOptimized.treeString).mkString("\n")))
}
- def structuralIntegrityOfInputPlanIsBrokenInClassError(className: String):
Throwable = {
- new RuntimeException("The structural integrity of the input plan is broken
in " +
- s"$className.")
+ def structuralIntegrityOfInputPlanIsBrokenInClassError(
+ className: String): SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2173",
+ messageParameters = Map(
+ "className" -> className))
}
def structuralIntegrityIsBrokenAfterApplyingRuleError(
- ruleName: String, batchName: String): Throwable = {
- new RuntimeException(s"After applying rule $ruleName in batch $batchName,
" +
- "the structural integrity of the plan is broken.")
+ ruleName: String, batchName: String): SparkRuntimeException = {
+ new SparkRuntimeException(
+ errorClass = "_LEGACY_ERROR_TEMP_2174",
+ messageParameters = Map(
+ "ruleName" -> ruleName,
+ "batchName" -> batchName))
}
def ruleIdNotFoundForRuleError(ruleName: String): Throwable = {
- new NoSuchElementException(s"Rule id not found for $ruleName")
+ new SparkException(
+ errorClass = "_LEGACY_ERROR_TEMP_2175",
+ messageParameters = Map(
+ "ruleName" -> ruleName),
+ cause = null)
}
def cannotCreateArrayWithElementsExceedLimitError(
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]