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
36 changes: 4 additions & 32 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -285,14 +285,14 @@
"The <exprName> must be between <valueRange> (current value = <currentValue>)"
]
},
"WRONG_NUM_ENDPOINTS" : {
"WRONG_NUM_ARGS" : {
Copy link
Member Author

Choose a reason for hiding this comment

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

@srielau FYI, I addressed your comment dece380#r87901056

"message" : [
"The number of endpoints must be >= 2 to construct intervals but the actual number is <actualNumber>."
"The <functionName> requires <expectedNum> parameters but the actual number is <actualNum>."
]
},
"WRONG_NUM_PARAMS" : {
"WRONG_NUM_ENDPOINTS" : {
"message" : [
"The <functionName> requires <expectedNum> parameters but the actual number is <actualNum>."
"The number of endpoints must be >= 2 to construct intervals but the actual number is <actualNumber>."
]
}
}
Expand Down Expand Up @@ -407,12 +407,6 @@
"Fail to recognize <pattern> pattern in the DateTimeFormatter. 1) You can set <config> to \"LEGACY\" to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html"
]
},
"FORMAT_DATETIME_BY_NEW_PARSER" : {
"message" : [
"Spark >= 3.0:",
"Fail to format it to <resultCandidate> in the new formatter. You can set <config> to \"LEGACY\" to restore the behavior before Spark 3.0, or set to \"CORRECTED\" and treat it as an invalid datetime string."
]
},
"PARSE_DATETIME_BY_NEW_PARSER" : {
"message" : [
"Spark >= 3.0:",
Expand Down Expand Up @@ -580,28 +574,6 @@
"More than one row returned by a subquery used as an expression."
]
},
"NAMESPACE_ALREADY_EXISTS" : {
"message" : [
"Cannot create namespace <nameSpaceName> because it already exists.",
"Choose a different name, drop the existing namespace, or add the IF NOT EXISTS clause to tolerate pre-existing namespace."
],
"sqlState" : "42000"
},
"NAMESPACE_NOT_EMPTY" : {
"message" : [
"Cannot drop a namespace <nameSpaceNameName> because it contains objects.",
"Use DROP NAMESPACE ... CASCADE to drop the namespace and all its objects."
],
"sqlState" : "42000"
},
"NAMESPACE_NOT_FOUND" : {
"message" : [
"The namespace <nameSpaceName> cannot be found. Verify the spelling and correctness of the namespace.",
"If you did not qualify the name with, verify the current_schema() output, or qualify the name with the correctly.",
"To tolerate the error on drop use DROP NAMESPACE IF EXISTS."
],
"sqlState" : "42000"
},
"NON_LITERAL_PIVOT_VALUES" : {
"message" : [
"Literal expressions required for pivot values, found <expression>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,5 +127,5 @@ class FunctionAlreadyExistsException(errorClass: String, messageParameters: Map[
}

class IndexAlreadyExistsException(message: String, cause: Option[Throwable] = None)
extends AnalysisException(errorClass = "INDEX_NOT_FOUND",
extends AnalysisException(errorClass = "INDEX_ALREADY_EXISTS",
Map("message" -> message), cause)
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ case class CallMethodViaReflection(children: Seq[Expression])
override def checkInputDataTypes(): TypeCheckResult = {
if (children.size < 2) {
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
"expectedNum" -> "> 1",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1209,7 +1209,7 @@ case class Least(children: Seq[Expression]) extends ComplexTypeMergingExpression
override def checkInputDataTypes(): TypeCheckResult = {
if (children.length <= 1) {
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
"expectedNum" -> "> 1",
Expand Down Expand Up @@ -1300,7 +1300,7 @@ case class Greatest(children: Seq[Expression]) extends ComplexTypeMergingExpress
override def checkInputDataTypes(): TypeCheckResult = {
if (children.length <= 1) {
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
"expectedNum" -> "> 1",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,7 @@ abstract class HashExpression[E] extends Expression {
override def checkInputDataTypes(): TypeCheckResult = {
if (children.length < 1) {
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
"expectedNum" -> "> 0",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -396,7 +396,7 @@ case class JsonTuple(children: Seq[Expression])
override def checkInputDataTypes(): TypeCheckResult = {
if (children.length < 2) {
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
"expectedNum" -> "> 1",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ case class Elt(
override def checkInputDataTypes(): TypeCheckResult = {
if (children.size < 2) {
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
"expectedNum" -> "> 1",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -978,7 +978,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def corruptedViewReferredTempFunctionsInCatalogError(e: Exception): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1088",
messageParameters = Map.empty,
cause = Some(e))
Expand Down Expand Up @@ -1309,19 +1309,19 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def tableIsNotRowLevelOperationTableError(table: Table): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1122",
messageParameters = Map("table" -> table.name()))
}

def cannotRenameTableWithAlterViewError(): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1123",
messageParameters = Map.empty)
}

private def notSupportedForV2TablesError(cmd: String): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1124",
messageParameters = Map("cmd" -> cmd))
}
Expand Down Expand Up @@ -1355,25 +1355,25 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def databaseFromV1SessionCatalogNotSpecifiedError(): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1125",
messageParameters = Map.empty)
}

def nestedDatabaseUnsupportedByV1SessionCatalogError(catalog: String): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1126",
messageParameters = Map("catalog" -> catalog))
}

def invalidRepartitionExpressionsError(sortOrders: Seq[Any]): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1127",
messageParameters = Map("sortOrders" -> sortOrders.toString()))
}

def partitionColumnNotSpecifiedError(format: String, partitionColumn: String): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1128",
messageParameters = Map(
"format" -> format,
Expand Down Expand Up @@ -2145,7 +2145,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
def invalidPatternError(pattern: String, message: String): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1216",
messageParameters = Map("pattern" -> pattern, "message" -> message))
messageParameters = Map(
"pattern" -> toSQLValue(pattern, StringType),
"message" -> message))
}

def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = {
Expand Down Expand Up @@ -2305,7 +2307,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def analyzeTableNotSupportedOnViewsError(): Throwable = {
new AnalysisException(
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1236",
messageParameters = Map.empty)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1395,7 +1395,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
def failToRecognizePatternError(pattern: String, e: Throwable): SparkRuntimeException = {
new SparkRuntimeException(
errorClass = "_LEGACY_ERROR_TEMP_2130",
messageParameters = Map("pattern" -> pattern),
messageParameters = Map("pattern" -> toSQLValue(pattern, StringType)),
cause = e)
}

Expand Down Expand Up @@ -2686,7 +2686,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
messageParameters = Map(
"parameter" -> "regexp",
"functionName" -> toSQLId(funcName),
"expected" -> pattern))
"expected" -> toSQLValue(pattern, StringType)))
}

def tooManyArrayElementsError(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer
exception = intercept[AnalysisException] {
assertSuccess(expr)
},
errorClass = "DATATYPE_MISMATCH.WRONG_NUM_PARAMS",
errorClass = "DATATYPE_MISMATCH.WRONG_NUM_ARGS",
parameters = messageParameters)
}

Expand Down Expand Up @@ -447,7 +447,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer
exception = intercept[AnalysisException] {
assertSuccess(murmur3Hash)
},
errorClass = "DATATYPE_MISMATCH.WRONG_NUM_PARAMS",
errorClass = "DATATYPE_MISMATCH.WRONG_NUM_ARGS",
parameters = Map(
"sqlExpr" -> "\"hash()\"",
"functionName" -> toSQLId(murmur3Hash.prettyName),
Expand All @@ -459,7 +459,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer
exception = intercept[AnalysisException] {
assertSuccess(xxHash64)
},
errorClass = "DATATYPE_MISMATCH.WRONG_NUM_PARAMS",
errorClass = "DATATYPE_MISMATCH.WRONG_NUM_ARGS",
parameters = Map(
"sqlExpr" -> "\"xxhash64()\"",
"functionName" -> toSQLId(xxHash64.prettyName),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -523,15 +523,15 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkExceptionInExpression[SparkRuntimeException](
RegExpExtract(s, p, r),
create_row("1a 2b 14m", "(?l)", 0),
s"$prefix `regexp_extract` is invalid: (?l)")
s"$prefix `regexp_extract` is invalid: '(?l)'")
checkExceptionInExpression[SparkRuntimeException](
RegExpExtractAll(s, p, r),
create_row("abc", "] [", 0),
s"$prefix `regexp_extract_all` is invalid: ] [")
s"$prefix `regexp_extract_all` is invalid: '] ['")
checkExceptionInExpression[SparkRuntimeException](
RegExpInStr(s, p, r),
create_row("abc", ", (", 0),
s"$prefix `regexp_instr` is invalid: , (")
s"$prefix `regexp_instr` is invalid: ', ('")
}

test("RegExpReplace: fails analysis if pos is not a constant") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1593,7 +1593,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val expr1 = Elt(Seq(indexExpr1))
assert(expr1.checkInputDataTypes() ==
DataTypeMismatch(
errorSubClass = "WRONG_NUM_PARAMS",
errorSubClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"functionName" -> "`elt`",
"expectedNum" -> "> 1",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -447,7 +447,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'a'",
"pattern" : "m%aca"
"pattern" : "'m%aca'"
}
}

Expand All @@ -462,7 +462,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'a'",
"pattern" : "m%aca"
"pattern" : "'m%aca'"
}
}

Expand All @@ -477,7 +477,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'a'",
"pattern" : "m%a%%a"
"pattern" : "'m%a%%a'"
}
}

Expand All @@ -492,7 +492,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'a'",
"pattern" : "m%a%%a"
"pattern" : "'m%a%%a'"
}
}

Expand All @@ -507,7 +507,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'e'",
"pattern" : "b_ear"
"pattern" : "'b_ear'"
}
}

Expand All @@ -522,7 +522,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'e'",
"pattern" : "b_ear"
"pattern" : "'b_ear'"
}
}

Expand All @@ -537,7 +537,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'e'",
"pattern" : "b_e__r"
"pattern" : "'b_e__r'"
}
}

Expand All @@ -552,7 +552,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1216",
"messageParameters" : {
"message" : "the escape character is not allowed to precede 'e'",
"pattern" : "b_e__r"
"pattern" : "'b_e__r'"
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ org.apache.spark.SparkRuntimeException
"errorClass" : "INVALID_PARAMETER_VALUE",
"sqlState" : "22023",
"messageParameters" : {
"expected" : "(?l)",
"expected" : "'(?l)'",
"functionName" : "`regexp_extract`",
"parameter" : "regexp"
}
Expand Down Expand Up @@ -334,7 +334,7 @@ org.apache.spark.SparkRuntimeException
"errorClass" : "INVALID_PARAMETER_VALUE",
"sqlState" : "22023",
"messageParameters" : {
"expected" : "], [",
"expected" : "'], ['",
"functionName" : "`regexp_extract_all`",
"parameter" : "regexp"
}
Expand Down Expand Up @@ -671,7 +671,7 @@ org.apache.spark.SparkRuntimeException
"errorClass" : "INVALID_PARAMETER_VALUE",
"sqlState" : "22023",
"messageParameters" : {
"expected" : ") ?",
"expected" : "') ?'",
"functionName" : "`regexp_instr`",
"parameter" : "regexp"
}
Expand Down
Loading