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
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB
// Early checks for column definitions, to produce better error messages
ColumnDefinition.checkColumnDefinitions(operator)

var stagedError: Option[() => Unit] = None
getAllExpressions(operator).foreach(_.foreachUp {
case a: Attribute if !a.resolved =>
failUnresolvedAttribute(operator, a, "UNRESOLVED_COLUMN")
Expand Down Expand Up @@ -305,12 +306,14 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB
s"Cannot resolve the runtime replaceable expression ${toSQLExpr(e)}. " +
s"The replacement is unresolved: ${toSQLExpr(e.replacement)}.")

// `Grouping` and `GroupingID` are considered as of having lower priority than the other
// nodes which cause errors.
case g: Grouping =>
g.failAnalysis(
errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", messageParameters = Map.empty)
if (stagedError.isEmpty) stagedError = Some(() => g.failAnalysis(
errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", messageParameters = Map.empty))
case g: GroupingID =>
g.failAnalysis(
errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", messageParameters = Map.empty)
if (stagedError.isEmpty) stagedError = Some(() => g.failAnalysis(
errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", messageParameters = Map.empty))

case e: Expression if e.children.exists(_.isInstanceOf[WindowFunction]) &&
!e.isInstanceOf[WindowExpression] && e.resolved =>
Expand Down Expand Up @@ -369,6 +372,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB

case _ =>
})
if (stagedError.isDefined) stagedError.get.apply()

operator match {
case RelationTimeTravel(u: UnresolvedRelation, _, _) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -926,6 +926,37 @@ class QueryCompilationErrorsSuite
parameters = Map("message" -> "Cannot convert Spark data type \"DUMMY\" to any Parquet type.")
)
}

test("SPARK-48556: Ensure UNRESOLVED_COLUMN is thrown when query has grouping expressions " +
"with invalid column name") {
case class UnresolvedDummyColumnTest(query: String, pos: Int)

withTable("t1") {
sql("create table t1(a int, b int) using parquet")
val tests = Seq(
UnresolvedDummyColumnTest("select grouping(a), dummy from t1 group by a with rollup", 20),
UnresolvedDummyColumnTest("select dummy, grouping(a) from t1 group by a with rollup", 7),
UnresolvedDummyColumnTest(
"select a, case when grouping(a) = 1 then 0 else b end, count(dummy) from t1 " +
"group by 1 with rollup",
61),
UnresolvedDummyColumnTest(
"select a, max(dummy), case when grouping(a) = 1 then 0 else b end " +
"from t1 group by 1 with rollup",
14)
)
tests.foreach(test => {
checkError(
exception = intercept[AnalysisException] {
sql(test.query)
},
errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
parameters = Map("objectName" -> "`dummy`", "proposal" -> "`a`, `b`"),
context = ExpectedContext(fragment = "dummy", start = test.pos, stop = test.pos + 4)
)
})
}
}
}

class MyCastToString extends SparkUserDefinedFunction(
Expand Down