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
8 changes: 7 additions & 1 deletion core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,12 @@
],
"sqlState" : "42000"
},
"UNRESOLVED_MAP_KEY" : {
"message" : [
"Cannot resolve column <columnName> as a map key. If the key is a string literal, please add single quotes around it. Otherwise, did you mean one of the following column(s)? [<proposal>]"
],
"sqlState" : "42000"
},
"UNSUPPORTED_DATATYPE" : {
"message" : [
"Unsupported data type <typeName>"
Expand Down Expand Up @@ -556,4 +562,4 @@
],
"sqlState" : "40000"
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -3420,8 +3420,8 @@ class Analyzer(override val catalogManager: CatalogManager)

i.userSpecifiedCols.map { col =>
i.table.resolve(Seq(col), resolver).getOrElse(
throw QueryCompilationErrors.unresolvedColumnError(
col, i.table.output.map(_.name), i.origin))
throw QueryCompilationErrors.unresolvedAttributeError(
"UNRESOLVED_COLUMN", col, i.table.output.map(_.name), i.origin))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,26 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
}
}

private def isMapWithStringKey(e: Expression): Boolean = if (e.resolved) {
e.dataType match {
case m: MapType => m.keyType.isInstanceOf[StringType]
case _ => false
}
} else {
false
}

private def failUnresolvedAttribute(
operator: LogicalPlan,
a: Attribute,
errorClass: String): Nothing = {
val missingCol = a.sql
val candidates = operator.inputSet.toSeq.map(_.qualifiedName)
val orderedCandidates = StringUtils.orderStringsBySimilarity(missingCol, candidates)
throw QueryCompilationErrors.unresolvedAttributeError(
errorClass, missingCol, orderedCandidates, a.origin)
}

def checkAnalysis(plan: LogicalPlan): Unit = {
// We transform up and order the rules so as to catch the first possible failure instead
// of the result of cascading resolution failures. Inline all CTEs in the plan to help check
Expand Down Expand Up @@ -160,27 +180,28 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
throw QueryCompilationErrors.commandUnsupportedInV2TableError("SHOW TABLE EXTENDED")

case operator: LogicalPlan =>
// Check argument data types of higher-order functions downwards first.
// If the arguments of the higher-order functions are resolved but the type check fails,
// the argument functions will not get resolved, but we should report the argument type
// check failure instead of claiming the argument functions are unresolved.
operator transformExpressionsDown {
// Check argument data types of higher-order functions downwards first.
// If the arguments of the higher-order functions are resolved but the type check fails,
// the argument functions will not get resolved, but we should report the argument type
// check failure instead of claiming the argument functions are unresolved.
case hof: HigherOrderFunction
if hof.argumentsResolved && hof.checkArgumentDataTypes().isFailure =>
hof.checkArgumentDataTypes() match {
case TypeCheckResult.TypeCheckFailure(message) =>
hof.failAnalysis(
s"cannot resolve '${hof.sql}' due to argument data type mismatch: $message")
}

// If an attribute can't be resolved as a map key of string type, either the key should be
// surrounded with single quotes, or there is a typo in the attribute name.
Copy link
Member Author

Choose a reason for hiding this comment

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

I tried to match element_at and try_element_at here. But before the functions are not resolved yet in the rule ResolveFunctions, the error MISSING_COLUMN is thrown.
Checking the unresolved function name and arguments can work but is quite ugly. I decided to handle the [] operator first.

case GetMapValue(map, key: Attribute, _) if isMapWithStringKey(map) && !key.resolved =>
failUnresolvedAttribute(operator, key, "UNRESOLVED_MAP_KEY")
}

getAllExpressions(operator).foreach(_.foreachUp {
case a: Attribute if !a.resolved =>
val missingCol = a.sql
val candidates = operator.inputSet.toSeq.map(_.qualifiedName)
val orderedCandidates = StringUtils.orderStringsBySimilarity(missingCol, candidates)
throw QueryCompilationErrors.unresolvedColumnError(
missingCol, orderedCandidates, a.origin)
failUnresolvedAttribute(operator, a, "UNRESOLVED_COLUMN")

case s: Star =>
withPosition(s) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -461,7 +461,7 @@ case class GetMapValue(
@transient private lazy val ordering: Ordering[Any] =
TypeUtils.getInterpretedOrdering(keyType)

private def keyType = child.dataType.asInstanceOf[MapType].keyType
private[catalyst] def keyType = child.dataType.asInstanceOf[MapType].keyType

override def checkInputDataTypes(): TypeCheckResult = {
super.checkInputDataTypes() match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,11 +144,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
s"side of the join. The $side-side columns: [${plan.output.map(_.name).mkString(", ")}]")
}

def unresolvedColumnError(
colName: String, candidates: Seq[String], origin: Origin): Throwable = {
def unresolvedAttributeError(
errorClass: String,
colName: String,
candidates: Seq[String],
origin: Origin): Throwable = {
val candidateIds = candidates.map(candidate => toSQLId(candidate))
new AnalysisException(
errorClass = "UNRESOLVED_COLUMN",
errorClass = errorClass,
messageParameters = Array(toSQLId(colName), candidateIds.mkString(", ")),
origin = origin)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -401,6 +401,18 @@ class QueryCompilationErrorsSuite
)
}

test("UNRESOLVED_MAP_KEY: string type literal should be quoted") {
checkAnswer(sql("select m['a'] from (select map('a', 'b') as m, 'aa' as aa)"), Row("b"))
checkError(
exception = intercept[AnalysisException] {
sql("select m[a] from (select map('a', 'b') as m, 'aa' as aa)")
Copy link
Contributor

Choose a reason for hiding this comment

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

In CheckAnalysis, how can we distinguish m.a and m[a]?

Copy link
Member Author

Choose a reason for hiding this comment

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

m.a is equivalent to m['a'] so it won't fail analysis

},
errorClass = "UNRESOLVED_MAP_KEY",
parameters = Map("columnName" -> "`a`",
"proposal" ->
"`__auto_generated_subquery_name`.`m`, `__auto_generated_subquery_name`.`aa`"))
}

test("UNRESOLVED_COLUMN: SELECT distinct does not work correctly " +
"if order by missing attribute") {
checkAnswer(
Expand Down