Skip to content

Commit

Permalink
[SPARK-6452] [SQL] Checks for missing attributes and unresolved opera…
Browse files Browse the repository at this point in the history
…tor for all types of operator

In `CheckAnalysis`, `Filter` and `Aggregate` are checked in separate case clauses, thus never hit those clauses for unresolved operators and missing input attributes.

This PR also removes the `prettyString` call when generating error message for missing input attributes. Because result of `prettyString` doesn't contain expression ID, and may give confusing messages like

> resolved attributes a missing from a

cc rxin

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/5129)
<!-- Reviewable:end -->

Author: Cheng Lian <[email protected]>

Closes #5129 from liancheng/spark-6452 and squashes the following commits:

52cdc69 [Cheng Lian] Addresses comments
029f9bd [Cheng Lian] Checks for missing attributes and unresolved operator for all types of operator
  • Loading branch information
liancheng authored and marmbrus committed Mar 24, 2015
1 parent 4ce2782 commit 1afcf77
Show file tree
Hide file tree
Showing 3 changed files with 33 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ class CheckAnalysis {
s"filter expression '${f.condition.prettyString}' " +
s"of type ${f.condition.dataType.simpleString} is not a boolean.")

case aggregatePlan@Aggregate(groupingExprs, aggregateExprs, child) =>
case Aggregate(groupingExprs, aggregateExprs, child) =>
def checkValidAggregateExpression(expr: Expression): Unit = expr match {
case _: AggregateExpression => // OK
case e: Attribute if !groupingExprs.contains(e) =>
Expand All @@ -85,13 +85,18 @@ class CheckAnalysis {

cleaned.foreach(checkValidAggregateExpression)

case _ => // Fallbacks to the following checks
}

operator match {
case o if o.children.nonEmpty && o.missingInput.nonEmpty =>
val missingAttributes = o.missingInput.map(_.prettyString).mkString(",")
val input = o.inputSet.map(_.prettyString).mkString(",")
val missingAttributes = o.missingInput.mkString(",")
val input = o.inputSet.mkString(",")

failAnalysis(s"resolved attributes $missingAttributes missing from $input")
failAnalysis(
s"resolved attribute(s) $missingAttributes missing from $input " +
s"in operator ${operator.simpleString}")

// Catch all
case o if !o.resolved =>
failAnalysis(
s"unresolved operator ${operator.simpleString}")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,9 +47,12 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy
* Attributes that are referenced by expressions but not provided by this nodes children.
* Subclasses should override this method if they produce attributes internally as it is used by
* assertions designed to prevent the construction of invalid plans.
*
* Note that virtual columns should be excluded. Currently, we only support the grouping ID
* virtual column.
*/
def missingInput: AttributeSet = (references -- inputSet)
.filter(_.name != VirtualColumn.groupingIdName)
def missingInput: AttributeSet =
(references -- inputSet).filter(_.name != VirtualColumn.groupingIdName)

/**
* Runs [[transform]] with `rule` on all expressions present in this query operator.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -199,4 +199,22 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
assert(pl(3).dataType == DecimalType.Unlimited)
assert(pl(4).dataType == DoubleType)
}

test("SPARK-6452 regression test") {
// CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s)
val plan =
Aggregate(
Nil,
Alias(Sum(AttributeReference("a", StringType)(exprId = ExprId(1))), "b")() :: Nil,
LocalRelation(
AttributeReference("a", StringType)(exprId = ExprId(2))))

assert(plan.resolved)

val message = intercept[AnalysisException] {
caseSensitiveAnalyze(plan)
}.getMessage

assert(message.contains("resolved attribute(s) a#1 missing from a#2"))
}
}

0 comments on commit 1afcf77

Please sign in to comment.