Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
bcd53d2
[SPARK-38692][SQL] Use error classes in the compilation errors of fun…
lvshaokang Apr 22, 2022
9602765
[SPARK-38692][SQL] Use error classes in the compilation errors of fun…
lvshaokang Apr 24, 2022
8e59f84
Merge branch 'master' into SPARK-38692
lvshaokang Apr 24, 2022
bea3fe5
[SPARK-38692][SQL] Use error classes in the compilation errors of fun…
lvshaokang Apr 24, 2022
efa8dac
[SPARK-38692][SQL] Use error classes in the compilation errors of fun…
lvshaokang Apr 24, 2022
85e4c96
[SPARK-38692][SQL] Use error classes in the compilation errors of fun…
lvshaokang Apr 25, 2022
e7c6a68
style(): code style format
lvshaokang Apr 26, 2022
bcd5a50
Merge branch 'master' into SPARK-38692
lvshaokang Apr 26, 2022
c397177
Merge branch 'master' into SPARK-38692
lvshaokang Apr 27, 2022
7c92a85
[SPARK-38692][SQL] Use error classes in the compilation errors of fun…
lvshaokang May 1, 2022
cb81ea5
Merge branch 'master' into SPARK-38692
lvshaokang May 1, 2022
e21b47a
style(): code style format
lvshaokang May 5, 2022
ddccd54
Merge branch 'master' into SPARK-38692
lvshaokang May 5, 2022
aaa0760
Merge branch 'master' into SPARK-38692
lvshaokang May 6, 2022
1106961
fix(): unify `INVALID_PARAMETER_VALUE` class and format message
lvshaokang Aug 17, 2022
ffc521b
Merge branch 'master' into SPARK-38692
lvshaokang Aug 17, 2022
c522bb8
fix(): format message
lvshaokang Aug 18, 2022
cd91055
Merge branch 'master' into SPARK-38692
lvshaokang Aug 18, 2022
82a6fd7
fix(): format message
lvshaokang Aug 18, 2022
acad322
fix(): format message
lvshaokang Aug 18, 2022
e84cbec
Merge branch 'master' into SPARK-38692
lvshaokang Aug 19, 2022
43ac9a0
format message with json
lvshaokang Aug 19, 2022
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
64 changes: 58 additions & 6 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -255,6 +255,64 @@
],
"sqlState" : "22023"
},
"INVALID_FUNCTION_ARGUMENTS" : {
Copy link
Member

Choose a reason for hiding this comment

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

Would it be possible to use the existing error class INVALID_PARAMETER_VALUE?

"message" : [
"Arguments of the <funcName> function are invalid:"
],
"subClass" : {
"APPROX_COUNT_DISTINCT" : {
"message" : [
"The second argument should be a double literal"
]
},
"CAST_ALIAS" : {
"message" : [
"Function accepts only one argument"
]
},
"EMPTY_NUMBER_OF_ARGUMENTS" : {
"message" : [
"Empty number of arguments"
]
},
"FIRST_LAST" : {
"message" : [
"The second argument should be a boolean literal"
]
},
"INVALID_ARGUMENT_INDEX" : {
"message" : [
"The value of parameter(s) '<parameter>' has invalid index, expects <expected>, but got <found>."
]
},
"INVALID_ARGUMENT_LENGTH" : {
"message" : [
"The value of parameter(s) '<parameter>' has invalid length, expects <expected>, but got <found>."
]
},
"INVALID_ARGUMENT_VALUE" : {
"message" : [
"The value of parameter(s) '<parameter>' is invalid: <message>"
]
},
"INVALID_NUMBER_OF_ARGUMENTS" : {
"message" : [
"Invalid number of arguments. Expected: <expected>; Found: <found>"
]
},
"INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION" : {
"message" : [
"There are <expected> arguments in V2Function, but <found> parameters returned from 'inputTypes()'"
]
},
"INVALID_OPERATION_FOR_V2FUNCTION" : {
"message" : [
"V2Function cannot process input: (<type>): <message>"
]
}
},
"sqlState" : "22023"
},
"INVALID_JSON_SCHEMA_MAP_TYPE" : {
"message" : [
"Input schema <jsonSchema> can only contain STRING as a key type for a MAP."
Expand All @@ -265,12 +323,6 @@
"The group aggregate pandas UDF <functionList> cannot be invoked together with as other, non-pandas aggregate functions."
]
},
"INVALID_PARAMETER_VALUE" : {
"message" : [
"The value of parameter(s) '<parameter>' in <functionName> is invalid: <expected>"
],
"sqlState" : "22023"
},
"INVALID_PROPERTY_KEY" : {
"message" : [
"<key> is an invalid property key, please use quotes, e.g. SET <key>=<value>"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,18 @@ class AnalysisException protected[sql] (
errorSubClass = Some(errorSubClass),
messageParameters = messageParameters)

def this(
errorClass: String,
errorSubClass: String,
messageParameters: Array[String],
cause: Option[Throwable]) =
this(
SparkThrowableHelper.getMessage(errorClass, errorSubClass, messageParameters),
errorClass = Some(errorClass),
errorSubClass = Some(errorSubClass),
messageParameters = messageParameters,
cause = cause)

def this(
errorClass: String,
errorSubClass: String,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,11 @@ object FunctionRegistryBase {
} catch {
// the exception is an invocation exception. To get a meaningful message, we need the
// cause.
case e: Exception => throw new AnalysisException(e.getCause.getMessage)
case e: Exception =>
e.getCause match {
case ae: AnalysisException if ae.errorClass.isDefined => throw ae;
case other => throw new AnalysisException(other.getMessage)
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,8 @@ case class ApproxCountDistinctForIntervals(
this(
child = child,
endpointsExpression = endpointsExpression,
relativeSD = HyperLogLogPlusPlus.validateDoubleLiteral(relativeSD),
relativeSD = HyperLogLogPlusPlus
.validateDoubleLiteral("ApproxCountDistinctForIntervals", relativeSD),
mutableAggBufferOffset = 0,
inputAggBufferOffset = 0)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ case class HyperLogLogPlusPlus(
def this(child: Expression, relativeSD: Expression) = {
this(
child = child,
relativeSD = HyperLogLogPlusPlus.validateDoubleLiteral(relativeSD),
relativeSD = HyperLogLogPlusPlus.validateDoubleLiteral("approx_count_distinct", relativeSD),
mutableAggBufferOffset = 0,
inputAggBufferOffset = 0)
}
Expand Down Expand Up @@ -144,10 +144,10 @@ case class HyperLogLogPlusPlus(
}

object HyperLogLogPlusPlus {
def validateDoubleLiteral(exp: Expression): Double = exp match {
def validateDoubleLiteral(name: String, exp: Expression): Double = exp match {
case Literal(d: Double, DoubleType) => d
case Literal(dec: Decimal, _) => dec.toDouble
case _ =>
throw QueryCompilationErrors.secondArgumentNotDoubleLiteralError
throw QueryCompilationErrors.secondArgumentNotDoubleLiteralError(name)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,9 +69,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {

def zeroArgumentIndexError(): Throwable = {
new AnalysisException(
errorClass = "INVALID_PARAMETER_VALUE",
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_ARGUMENT_INDEX",
messageParameters = Array(
"strfmt", toSQLId("format_string"), "expects %1$, %2$ and so on, but got %0$."))
toSQLId("format_string"), "strfmt", "%1$, %2$ and so on", "%0$"))
}

def unorderablePivotColError(pivotCol: Expression): Throwable = {
Expand Down Expand Up @@ -496,14 +497,19 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {

def invalidFunctionArgumentsError(
name: String, expectedInfo: String, actualNumber: Int): Throwable = {
new AnalysisException(s"Invalid number of arguments for function $name. " +
s"Expected: $expectedInfo; Found: $actualNumber")
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_NUMBER_OF_ARGUMENTS",
messageParameters = Array(name, expectedInfo, s"$actualNumber"))
}

def invalidFunctionArgumentNumberError(
validParametersCount: Seq[Int], name: String, actualNumber: Int): Throwable = {
if (validParametersCount.length == 0) {
new AnalysisException(s"Invalid arguments for function $name")
if (validParametersCount.isEmpty) {
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "EMPTY_NUMBER_OF_ARGUMENTS",
messageParameters = Array(name))
} else {
val expectedNumberOfParameters = if (validParametersCount.length == 1) {
validParametersCount.head.toString
Expand All @@ -516,7 +522,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def functionAcceptsOnlyOneArgumentError(name: String): Throwable = {
new AnalysisException(s"Function $name accepts only one argument")
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "CAST_ALIAS",
messageParameters = Array(name))
}

def alterV2TableSetLocationWithPartitionNotSupportedError(): Throwable = {
Expand Down Expand Up @@ -819,8 +828,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
new AnalysisException(s"Unsupported component type $clz in arrays")
}

def secondArgumentNotDoubleLiteralError(): Throwable = {
new AnalysisException("The second argument should be a double literal.")
def secondArgumentNotDoubleLiteralError(name: String): Throwable = {
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "APPROX_COUNT_DISTINCT",
messageParameters = Array(name)
)
}

def dataTypeUnsupportedByExtractValueError(
Expand Down Expand Up @@ -1540,16 +1553,26 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
unbound: UnboundFunction,
arguments: Seq[Expression],
unsupported: UnsupportedOperationException): Throwable = {
new AnalysisException(s"Function '${unbound.name}' cannot process " +
s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
unsupported.getMessage, cause = Some(unsupported))
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_OPERATION_FOR_V2FUNCTION",
messageParameters = Array(
unbound.name,
arguments.map(x => toSQLType(x.dataType)).mkString(", "),
unsupported.getMessage),
cause = Some(unsupported))
}

def v2FunctionInvalidInputTypeLengthError(
bound: BoundFunction,
args: Seq[Expression]): Throwable = {
new AnalysisException(s"Invalid bound function '${bound.name()}: there are ${args.length} " +
s"arguments but ${bound.inputTypes().length} parameters returned from 'inputTypes()'")
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION",
messageParameters = Array(
bound.name(), s"${args.length}",
s"${bound.inputTypes().length}")
)
}

def ambiguousRelationAliasNameInNestedCTEError(name: String): Throwable = {
Expand Down Expand Up @@ -1629,7 +1652,11 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def secondArgumentInFunctionIsNotBooleanLiteralError(funcName: String): Throwable = {
new AnalysisException(s"The second argument in $funcName should be a boolean literal.")
new AnalysisException(
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "FIRST_LAST",
messageParameters = Array(funcName)
)
}

def joinConditionMissingOrTrivialError(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1981,11 +1981,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {

def invalidAesKeyLengthError(actualLength: Int): RuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_PARAMETER_VALUE",
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_ARGUMENT_LENGTH",
messageParameters = Array(
"key",
aesFuncName,
s"expects a binary value with 16, 24 or 32 bytes, but got ${actualLength.toString} bytes."))
"key",
"a binary value with 16, 24 or 32 bytes",
s"${actualLength.toString} bytes"))
}

def aesModeUnsupportedError(mode: String, padding: String): RuntimeException = {
Expand All @@ -1997,10 +1999,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {

def aesCryptoError(detailMessage: String): RuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_PARAMETER_VALUE",
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_ARGUMENT_VALUE",
messageParameters = Array(
"expr, key",
aesFuncName,
"expr, key",
s"Detail message: $detailMessage"))
}

Expand Down Expand Up @@ -2083,10 +2086,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {

def invalidPatternError(funcName: String, pattern: String): RuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_PARAMETER_VALUE",
errorClass = "INVALID_FUNCTION_ARGUMENTS",
errorSubClass = "INVALID_ARGUMENT_VALUE",
messageParameters = Array(
"regexp",
toSQLId(funcName),
"regexp",
pattern))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -502,18 +502,20 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val s = $"s".string.at(0)
val p = $"p".string.at(1)
val r = $"r".int.at(2)
val prefix = "[INVALID_PARAMETER_VALUE] The value of parameter(s) 'regexp' in"
val prefix = "[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] " +
"Arguments of the %s function are invalid: " +
"The value of parameter(s) 'regexp'"
checkExceptionInExpression[SparkRuntimeException](
RegExpExtract(s, p, r),
create_row("1a 2b 14m", "(?l)", 0),
s"$prefix `regexp_extract` is invalid: (?l)")
s"${prefix.format("`regexp_extract`")} is invalid: (?l)")
checkExceptionInExpression[SparkRuntimeException](
RegExpExtractAll(s, p, r),
create_row("abc", "] [", 0),
s"$prefix `regexp_extract_all` is invalid: ] [")
s"${prefix.format("`regexp_extract_all`")} is invalid: ] [")
checkExceptionInExpression[SparkRuntimeException](
RegExpInStr(s, p, r),
create_row("abc", ", (", 0),
s"$prefix `regexp_instr` is invalid: , (")
s"${prefix.format("`regexp_instr`")} is invalid: , (")
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -112,14 +112,14 @@ class FirstLastTestSuite extends SparkFunSuite {
val msg1 = intercept[AnalysisException] {
new First(input, Literal(1, IntegerType))
}.getMessage
assert(msg1.contains("The second argument in first should be a boolean literal"))
assert(msg1.contains("The second argument should be a boolean literal"))
val msg2 = intercept[AnalysisException] {
new Last(input, Literal(1, IntegerType))
}.getMessage
assert(msg2.contains("The second argument in last should be a boolean literal"))
assert(msg2.contains("The second argument should be a boolean literal"))
val msg3 = intercept[AnalysisException] {
new AnyValue(input, Literal(1, IntegerType))
}.getMessage
assert(msg3.contains("The second argument in any_value should be a boolean literal"))
assert(msg3.contains("The second argument should be a boolean literal"))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -719,17 +719,33 @@ select decode()
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7

{
"errorClass" : "INVALID_FUNCTION_ARGUMENTS",
"errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS",
"sqlState" : "22023",
"messageParameters" : {
"funcName" : "decode",
"expected" : "2",
"found" : "0"
}
}

-- !query
select decode(encode('abc', 'utf-8'))
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7

{
"errorClass" : "INVALID_FUNCTION_ARGUMENTS",
"errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS",
"sqlState" : "22023",
"messageParameters" : {
"funcName" : "decode",
"expected" : "2",
"found" : "1"
}
}

-- !query
select decode(encode('abc', 'utf-8'), 'utf-8')
Expand Down
Loading