aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorMichael Armbrust <michael@databricks.com>2015-02-10 17:32:42 -0800
committerMichael Armbrust <michael@databricks.com>2015-02-10 17:32:42 -0800
commit6195e2473b98253ccc9edc3d624ba2bf59ffc398 (patch)
tree33afc671375362f703028c8c96521198cb57cc17 /sql
parentaaf50d05c7616e4f8f16654b642500ae06cdd774 (diff)
downloadspark-6195e2473b98253ccc9edc3d624ba2bf59ffc398.tar.gz
spark-6195e2473b98253ccc9edc3d624ba2bf59ffc398.tar.bz2
spark-6195e2473b98253ccc9edc3d624ba2bf59ffc398.zip
[SQL] Add an exception for analysis errors.
Also start from the bottom so we show the first error instead of the top error. Author: Michael Armbrust <michael@databricks.com> Closes #4439 from marmbrus/analysisException and squashes the following commits: 45862a0 [Michael Armbrust] fix hive test a773bba [Michael Armbrust] Merge remote-tracking branch 'origin/master' into analysisException f88079f [Michael Armbrust] update more cases fede90a [Michael Armbrust] newline fbf4bc3 [Michael Armbrust] move to sql 6235db4 [Michael Armbrust] [SQL] Add an exception for analysis errors.
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala23
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala21
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala14
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala3
5 files changed, 46 insertions, 17 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
new file mode 100644
index 0000000000..871d560b9d
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+/**
+ * Thrown when a query fails to analyze, usually because the query itself is invalid.
+ */
+class AnalysisException(message: String) extends Exception(message) with Serializable
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 fb2ff014ce..3f0d77ad63 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
@@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.util.collection.OpenHashSet
+import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
@@ -80,16 +81,18 @@ class Analyzer(catalog: Catalog,
*/
object CheckResolution extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = {
- plan.transform {
+ plan.transformUp {
case p if p.expressions.exists(!_.resolved) =>
- throw new TreeNodeException(p,
- s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}")
+ val missing = p.expressions.filterNot(_.resolved).map(_.prettyString).mkString(",")
+ val from = p.inputSet.map(_.name).mkString("{", ", ", "}")
+
+ throw new AnalysisException(s"Cannot resolve '$missing' given input columns $from")
case p if !p.resolved && p.childrenResolved =>
- throw new TreeNodeException(p, "Unresolved plan found")
+ throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}")
} match {
// As a backstop, use the root node to check that the entire plan tree is resolved.
case p if !p.resolved =>
- throw new TreeNodeException(p, "Unresolved plan in tree")
+ throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}")
case p => p
}
}
@@ -314,10 +317,11 @@ class Analyzer(catalog: Catalog,
val checkField = (f: StructField) => resolver(f.name, fieldName)
val ordinal = fields.indexWhere(checkField)
if (ordinal == -1) {
- sys.error(
+ throw new AnalysisException(
s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}")
} else if (fields.indexWhere(checkField, ordinal + 1) != -1) {
- sys.error(s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}")
+ throw new AnalysisException(
+ s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}")
} else {
ordinal
}
@@ -329,7 +333,8 @@ class Analyzer(catalog: Catalog,
case ArrayType(StructType(fields), containsNull) =>
val ordinal = findField(fields)
ArrayGetField(expr, fields(ordinal), ordinal, containsNull)
- case otherType => sys.error(s"GetField is not valid on fields of type $otherType")
+ case otherType =>
+ throw new AnalysisException(s"GetField is not valid on fields of type $otherType")
}
}
}
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 60060bf029..f011a5ff15 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,8 +19,8 @@ package org.apache.spark.sql.catalyst.analysis
import org.scalatest.{BeforeAndAfter, FunSuite}
+import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference}
-import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.types._
@@ -69,12 +69,12 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) ===
Project(testRelation.output, testRelation))
- val e = intercept[TreeNodeException[_]] {
+ val e = intercept[AnalysisException] {
caseSensitiveAnalyze(
Project(Seq(UnresolvedAttribute("tBl.a")),
UnresolvedRelation(Seq("TaBlE"), Some("TbL"))))
}
- assert(e.getMessage().toLowerCase.contains("unresolved"))
+ assert(e.getMessage().toLowerCase.contains("cannot resolve"))
assert(
caseInsensitiveAnalyze(
@@ -109,10 +109,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
}
test("throw errors for unresolved attributes during analysis") {
- val e = intercept[TreeNodeException[_]] {
+ val e = intercept[AnalysisException] {
caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation))
}
- assert(e.getMessage().toLowerCase.contains("unresolved attribute"))
+ assert(e.getMessage().toLowerCase.contains("cannot resolve"))
}
test("throw errors for unresolved plans during analysis") {
@@ -120,10 +120,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
override lazy val resolved = false
override def output = Nil
}
- val e = intercept[TreeNodeException[_]] {
+ val e = intercept[AnalysisException] {
caseSensitiveAnalyze(UnresolvedTestPlan())
}
- assert(e.getMessage().toLowerCase.contains("unresolved plan"))
+ assert(e.getMessage().toLowerCase.contains("unresolved"))
}
test("divide should be casted into fractional types") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 11502edf97..55fd0b0892 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -589,7 +589,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Row(_)))
// Column type mismatches where a coercion is not possible, in this case between integer
// and array types, trigger a TreeNodeException.
- intercept[TreeNodeException[_]] {
+ intercept[AnalysisException] {
sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect()
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index ff8130ae5f..ab5f9cdddf 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.hive.execution
+import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.hive.test.TestHive.{sparkContext, jsonRDD, sql}
import org.apache.spark.sql.hive.test.TestHive.implicits._
@@ -40,7 +41,7 @@ class HiveResolutionSuite extends HiveComparisonTest {
"""{"a": [{"b": 1, "B": 2}]}""" :: Nil)).registerTempTable("nested")
// there are 2 filed matching field name "b", we should report Ambiguous reference error
- val exception = intercept[RuntimeException] {
+ val exception = intercept[AnalysisException] {
sql("SELECT a[0].b from nested").queryExecution.analyzed
}
assert(exception.getMessage.contains("Ambiguous reference to fields"))