You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2014/07/04 09:53:44 UTC

git commit: [SPARK-2059][SQL] Add analysis checks

Repository: spark
Updated Branches:
  refs/heads/master 0bbe61223 -> b3e768e15


[SPARK-2059][SQL] Add analysis checks

This replaces #1263 with a test case.

Author: Reynold Xin <rx...@apache.org>
Author: Michael Armbrust <mi...@databricks.com>

Closes #1265 from rxin/sql-analysis-error and squashes the following commits:

a639e01 [Reynold Xin] Added a test case for unresolved attribute analysis.
7371e1b [Reynold Xin] Merge pull request #1263 from marmbrus/analysisChecks
448c088 [Michael Armbrust] Add analysis checks


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b3e768e1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b3e768e1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b3e768e1

Branch: refs/heads/master
Commit: b3e768e154bd7175db44c3ffc3d8f783f15ab776
Parents: 0bbe612
Author: Reynold Xin <rx...@apache.org>
Authored: Fri Jul 4 00:53:41 2014 -0700
Committer: Reynold Xin <rx...@apache.org>
Committed: Fri Jul 4 00:53:41 2014 -0700

----------------------------------------------------------------------
 .../spark/sql/catalyst/analysis/Analyzer.scala      | 16 ++++++++++++++++
 .../spark/sql/catalyst/analysis/AnalysisSuite.scala |  8 ++++++++
 2 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/b3e768e1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 4ebc0e7..c718846 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
+import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
@@ -54,11 +55,26 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
       ResolveFunctions ::
       GlobalAggregates ::
       typeCoercionRules :_*),
+    Batch("Check Analysis", Once,
+      CheckResolution),
     Batch("AnalysisOperators", fixedPoint,
       EliminateAnalysisOperators)
   )
 
   /**
+   * Makes sure all attributes have been resolved.
+   */
+  object CheckResolution extends Rule[LogicalPlan] {
+    def apply(plan: LogicalPlan): LogicalPlan = {
+      plan.transform {
+        case p if p.expressions.exists(!_.resolved) =>
+          throw new TreeNodeException(p,
+            s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}")
+      }
+    }
+  }
+
+  /**
    * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
    */
   object ResolveRelations extends Rule[LogicalPlan] {

http://git-wip-us.apache.org/repos/asf/spark/blob/b3e768e1/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 4c31358..f14df81 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis
 
 import org.scalatest.FunSuite
 
+import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.plans.logical._
 
 /* Implicit conversions */
@@ -34,4 +35,11 @@ class AnalysisSuite extends FunSuite {
       analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) ===
         Project(testRelation.output, testRelation))
   }
+
+  test("throw errors for unresolved attributes during analysis") {
+    val e = intercept[TreeNodeException[_]] {
+      analyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation))
+    }
+    assert(e.getMessage().toLowerCase.contains("unresolved"))
+  }
 }