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
114 changes: 57 additions & 57 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -229,46 +229,6 @@
"Input to <functionName> should all be the same type, but it's <dataType>."
]
},
"FORMAT_CONT_THOUSANDS_SEPS" : {
"message" : [
"Thousands separators (, or G) must have digits in between them in the number format: <format>."
]
},
"FORMAT_CUR_MUST_BEFORE_DEC" : {
"message" : [
"Currency characters must appear before any decimal point in the number format: <format>."
]
},
"FORMAT_CUR_MUST_BEFORE_DIGIT" : {
"message" : [
"Currency characters must appear before digits in the number format: <format>."
]
},
"FORMAT_EMPTY" : {
"message" : [
"The number format string cannot be empty."
]
},
"FORMAT_THOUSANDS_SEPS_MUST_BEFORE_DEC" : {
"message" : [
"Thousands separators (, or G) may not appear after the decimal point in the number format: <format>."
]
},
"FORMAT_UNEXPECTED_TOKEN" : {
"message" : [
"Unexpected <token> found in the format string <format>; the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]."
]
},
"FORMAT_WRONG_NUM_DIGIT" : {
"message" : [
"The format string requires at least one number digit."
]
},
"FORMAT_WRONG_NUM_TOKEN" : {
"message" : [
"At most one <token> is allowed in the number format: <format>."
]
},
"HASH_MAP_TYPE" : {
"message" : [
"Input to the function <functionName> cannot contain elements of the \"MAP\" type. In Spark, same maps may have different hashcode, thus hash expressions are prohibited on \"MAP\" elements. To restore previous behavior set \"spark.sql.legacy.allowHashOnMapType\" to \"true\"."
Expand Down Expand Up @@ -687,6 +647,63 @@
],
"sqlState" : "42000"
},
"INVALID_FORMAT" : {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we want this to be generic or specific to numbers.
I'm specifically wondering about DATE FORMAT, INTERVAL FORMAT, .. What have we/will we do with these?

Copy link
Contributor

Choose a reason for hiding this comment

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

Basically: Should we use INVALID_NUMBER_FORMAT, INVALID_DATETIME_FROMAT, .... or just this and bring them all in?

Copy link
Member Author

Choose a reason for hiding this comment

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

Do we want this to be generic or specific to numbers.

I want to believe that we can make it as a generic error class. I have already merged LIKE ... errors into it.

I think there is the common part where we declare that the format is invalid and show it.

"message" : [
"The format is invalid: <format>."
],
"subClass" : {
"CONT_THOUSANDS_SEPS" : {
"message" : [
"Thousands separators (, or G) must have digits in between them in the number format."
]
},
"CUR_MUST_BEFORE_DEC" : {
"message" : [
"Currency characters must appear before any decimal point in the number format."
]
},
"CUR_MUST_BEFORE_DIGIT" : {
"message" : [
"Currency characters must appear before digits in the number format."
]
},
"EMPTY" : {
"message" : [
"The number format string cannot be empty."
]
},
"ESC_AT_THE_END" : {
"message" : [
"The escape character is not allowed to end with."
]
},
"ESC_IN_THE_MIDDLE" : {
"message" : [
"The escape character is not allowed to precede <char>."
]
},
"THOUSANDS_SEPS_MUST_BEFORE_DEC" : {
"message" : [
"Thousands separators (, or G) may not appear after the decimal point in the number format."
]
},
"UNEXPECTED_TOKEN" : {
"message" : [
"Found the unexpected <token> in the format string; the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]."
]
},
"WRONG_NUM_DIGIT" : {
"message" : [
"The format string requires at least one number digit."
]
},
"WRONG_NUM_TOKEN" : {
"message" : [
"At most one <token> is allowed in the number format."
]
}
}
},
"INVALID_FRACTION_OF_SECOND" : {
"message" : [
"The fraction of sec must be zero. Valid range is [0, 60]. If necessary set <ansiConfig> to \"false\" to bypass this error."
Expand All @@ -708,23 +725,6 @@
"The <joinType> JOIN with LATERAL correlation is not allowed because an OUTER subquery cannot correlate to its join partner. Remove the LATERAL correlation or use an INNER JOIN, or LEFT OUTER JOIN instead."
]
},
"INVALID_LIKE_PATTERN" : {
"message" : [
"The pattern <pattern> is invalid."
],
"subClass" : {
"ESC_AT_THE_END" : {
"message" : [
"the escape character is not allowed to end with."
]
},
"ESC_IN_THE_MIDDLE" : {
"message" : [
"the escape character is not allowed to precede <char>."
]
}
}
},
"INVALID_PANDAS_UDF_PLACEMENT" : {
"message" : [
"The group aggregate pandas UDF <functionList> cannot be invoked together with as other, non-pandas aggregate functions."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB
val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil

val DATA_TYPE_MISMATCH_ERROR = TreeNodeTag[Boolean]("dataTypeMismatchError")
val INVALID_FORMAT_ERROR = TreeNodeTag[Boolean]("invalidFormatError")

/**
* Fails the analysis at the point where a specific tree node was parsed using a provided
Expand Down Expand Up @@ -221,6 +222,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB
hof.failAnalysis(
errorClass = "_LEGACY_ERROR_TEMP_2314",
messageParameters = Map("sqlExpr" -> hof.sql, "msg" -> message))
case checkRes: TypeCheckResult.InvalidFormat =>
hof.setTagValue(INVALID_FORMAT_ERROR, true)
hof.invalidFormat(checkRes)
}

// If an attribute can't be resolved as a map key of string type, either the key should be
Expand Down Expand Up @@ -252,6 +256,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB
"sqlExpr" -> e.sql,
"msg" -> message,
"hint" -> extraHintForAnsiTypeCoercionExpression(operator)))
case checkRes: TypeCheckResult.InvalidFormat =>
e.setTagValue(INVALID_FORMAT_ERROR, true)
e.invalidFormat(checkRes)
}

case c: Cast if !c.resolved =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,4 +55,17 @@ object TypeCheckResult {
extends TypeCheckResult {
def isSuccess: Boolean = false
}

/**
* Represents an error of invalid format with the `INVALID_FORMAT` error class.
*
* @param errorSubClass A sub-class of `INVALID_FORMAT`.
* @param messageParameters Parameters of the sub-class error message.
*/
case class InvalidFormat(
errorSubClass: String,
messageParameters: Map[String, String] = Map.empty)
extends TypeCheckResult {
def isSuccess: Boolean = false
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, InvalidFormat}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.util.quoteNameParts
Expand Down Expand Up @@ -73,6 +73,13 @@ package object analysis {
origin = t.origin)
}

def invalidFormat(invalidFormat: InvalidFormat): Nothing = {
throw new AnalysisException(
errorClass = s"INVALID_FORMAT.${invalidFormat.errorSubClass}",
messageParameters = invalidFormat.messageParameters,
origin = t.origin)
}

def tableNotFound(name: Seq[String]): Nothing = {
throw new AnalysisException(
errorClass = "TABLE_OR_VIEW_NOT_FOUND",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.util
import scala.collection.mutable

import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{InvalidFormat, TypeCheckSuccess}
import org.apache.spark.sql.catalyst.expressions.Cast._
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.types.{Decimal, DecimalType, StringType}
Expand Down Expand Up @@ -281,37 +281,29 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali

// Make sure the format string contains at least one token.
if (numberFormat.isEmpty) {
return DataTypeMismatch(
errorSubClass = "FORMAT_EMPTY",
messageParameters = Map.empty
)
return InvalidFormat(
errorSubClass = "EMPTY",
messageParameters = Map("format" -> toSQLValue(numberFormat, StringType)))
}
// Make sure the format string contains at least one digit.
if (!formatTokens.exists(
token => token.isInstanceOf[DigitGroups])) {
return DataTypeMismatch(
errorSubClass = "FORMAT_WRONG_NUM_DIGIT",
messageParameters = Map.empty
)
return InvalidFormat(
errorSubClass = "WRONG_NUM_DIGIT",
messageParameters = Map("format" -> toSQLValue(numberFormat, StringType)))
}
// Make sure that any dollar sign in the format string occurs before any digits.
if (firstDigitIndex < firstDollarSignIndex) {
return DataTypeMismatch(
errorSubClass = "FORMAT_CUR_MUST_BEFORE_DIGIT",
messageParameters = Map(
"format" -> toSQLValue(numberFormat, StringType)
)
)
return InvalidFormat(
errorSubClass = "CUR_MUST_BEFORE_DIGIT",
messageParameters = Map("format" -> toSQLValue(numberFormat, StringType)))
}
// Make sure that any dollar sign in the format string occurs before any decimal point.
if (firstDecimalPointIndex != -1 &&
firstDecimalPointIndex < firstDollarSignIndex) {
return DataTypeMismatch(
errorSubClass = "FORMAT_CUR_MUST_BEFORE_DEC",
messageParameters = Map(
"format" -> toSQLValue(numberFormat, StringType)
)
)
return InvalidFormat(
errorSubClass = "CUR_MUST_BEFORE_DEC",
messageParameters = Map("format" -> toSQLValue(numberFormat, StringType)))
}
// Make sure that any thousands separators in the format string have digits before and after.
if (digitGroupsBeforeDecimalPoint.exists {
Expand All @@ -327,23 +319,18 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali
false
})
}) {
return DataTypeMismatch(
errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS",
messageParameters = Map(
"format" -> toSQLValue(numberFormat, StringType)
)
)
return InvalidFormat(
errorSubClass = "CONT_THOUSANDS_SEPS",
messageParameters = Map("format" -> toSQLValue(numberFormat, StringType)))
}
// Make sure that thousands separators does not appear after the decimal point, if any.
if (digitGroupsAfterDecimalPoint.exists {
case DigitGroups(tokens, digits) =>
tokens.length > digits.length
}) {
return DataTypeMismatch(
errorSubClass = "FORMAT_THOUSANDS_SEPS_MUST_BEFORE_DEC",
messageParameters = Map(
"format" -> toSQLValue(numberFormat, StringType)
)
return InvalidFormat(
errorSubClass = "THOUSANDS_SEPS_MUST_BEFORE_DEC",
messageParameters = Map("format" -> toSQLValue(numberFormat, StringType))
)
}
// Make sure that the format string does not contain any prohibited duplicate tokens.
Expand All @@ -354,13 +341,11 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali
DollarSign(),
ClosingAngleBracket()).foreach {
token => if (inputTokenCounts.getOrElse(token, 0) > 1) {
return DataTypeMismatch(
errorSubClass = "FORMAT_WRONG_NUM_TOKEN",
return InvalidFormat(
errorSubClass = "WRONG_NUM_TOKEN",
messageParameters = Map(
"token" -> token.toString,
"format" -> toSQLValue(numberFormat, StringType)
)
)
"format" -> toSQLValue(numberFormat, StringType)))
}
}
// Enforce the ordering of tokens in the format string according to this specification:
Expand Down Expand Up @@ -393,13 +378,11 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali
}
}
if (formatTokenIndex < formatTokens.length) {
return DataTypeMismatch(
errorSubClass = "FORMAT_UNEXPECTED_TOKEN",
return InvalidFormat(
errorSubClass = "UNEXPECTED_TOKEN",
messageParameters = Map(
"token" -> formatTokens(formatTokenIndex).toString,
"format" -> toSQLValue(numberFormat, StringType)
)
)
"format" -> toSQLValue(numberFormat, StringType)))
}
// Validation of the format string finished successfully.
TypeCheckSuccess
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2143,17 +2143,16 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {

def escapeCharacterInTheMiddleError(pattern: String, char: String): Throwable = {
new AnalysisException(
errorClass = "INVALID_LIKE_PATTERN.ESC_IN_THE_MIDDLE",
errorClass = "INVALID_FORMAT.ESC_IN_THE_MIDDLE",
messageParameters = Map(
"pattern" -> toSQLValue(pattern, StringType),
"format" -> toSQLValue(pattern, StringType),
"char" -> toSQLValue(char, StringType)))
}

def escapeCharacterAtTheEndError(pattern: String): Throwable = {
new AnalysisException(
errorClass = "INVALID_LIKE_PATTERN.ESC_AT_THE_END",
messageParameters = Map(
"pattern" -> toSQLValue(pattern, StringType)))
errorClass = "INVALID_FORMAT.ESC_AT_THE_END",
messageParameters = Map("format" -> toSQLValue(pattern, StringType)))
}

def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,15 +154,18 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
// scalastyle:on nonascii

// invalid escaping
val invalidEscape = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like """\a""")
}
assert(invalidEscape.getMessage.contains("pattern"))

val endEscape = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like """a\""")
}
assert(endEscape.getMessage.contains("pattern"))
checkError(
exception = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like """\a""")
},
errorClass = "INVALID_FORMAT.ESC_IN_THE_MIDDLE",
parameters = Map("format" -> """'\\a'""", "char" -> "'a'"))
checkError(
exception = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like """a\""")
},
errorClass = "INVALID_FORMAT.ESC_AT_THE_END",
parameters = Map("format" -> """'a\\'"""))

// case
checkLiteralRow("A" like _, "a%", false)
Expand Down Expand Up @@ -231,14 +234,12 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
// scalastyle:on nonascii

// invalid escaping
val invalidEscape = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like(s"""${escapeChar}a""", escapeChar))
}
assert(invalidEscape.getMessage.contains("pattern"))
val endEscape = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like(s"""a$escapeChar""", escapeChar))
}
assert(endEscape.getMessage.contains("pattern"))
checkError(
exception = intercept[AnalysisException] {
evaluateWithoutCodegen("""a""" like(s"""${escapeChar}a""", escapeChar))
},
errorClass = "INVALID_FORMAT.ESC_IN_THE_MIDDLE",
parameters = Map("format" -> s"'${escapeChar}a'", "char" -> "'a'"))

// case
checkLiteralRow("A" like(_, escapeChar), "a%", false)
Expand Down
Loading