Skip to content

Commit 6195e24

Browse files
committed
[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.
1 parent aaf50d0 commit 6195e24

File tree

5 files changed

+46
-17
lines changed

5 files changed

+46
-17
lines changed
Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,23 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql
19+
20+
/**
21+
* Thrown when a query fails to analyze, usually because the query itself is invalid.
22+
*/
23+
class AnalysisException(message: String) extends Exception(message) with Serializable

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 13 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.spark.sql.catalyst.analysis
1919

2020
import org.apache.spark.util.collection.OpenHashSet
21+
import org.apache.spark.sql.AnalysisException
2122
import org.apache.spark.sql.catalyst.errors.TreeNodeException
2223
import org.apache.spark.sql.catalyst.expressions._
2324
import org.apache.spark.sql.catalyst.plans.logical._
@@ -80,16 +81,18 @@ class Analyzer(catalog: Catalog,
8081
*/
8182
object CheckResolution extends Rule[LogicalPlan] {
8283
def apply(plan: LogicalPlan): LogicalPlan = {
83-
plan.transform {
84+
plan.transformUp {
8485
case p if p.expressions.exists(!_.resolved) =>
85-
throw new TreeNodeException(p,
86-
s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}")
86+
val missing = p.expressions.filterNot(_.resolved).map(_.prettyString).mkString(",")
87+
val from = p.inputSet.map(_.name).mkString("{", ", ", "}")
88+
89+
throw new AnalysisException(s"Cannot resolve '$missing' given input columns $from")
8790
case p if !p.resolved && p.childrenResolved =>
88-
throw new TreeNodeException(p, "Unresolved plan found")
91+
throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}")
8992
} match {
9093
// As a backstop, use the root node to check that the entire plan tree is resolved.
9194
case p if !p.resolved =>
92-
throw new TreeNodeException(p, "Unresolved plan in tree")
95+
throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}")
9396
case p => p
9497
}
9598
}
@@ -314,10 +317,11 @@ class Analyzer(catalog: Catalog,
314317
val checkField = (f: StructField) => resolver(f.name, fieldName)
315318
val ordinal = fields.indexWhere(checkField)
316319
if (ordinal == -1) {
317-
sys.error(
320+
throw new AnalysisException(
318321
s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}")
319322
} else if (fields.indexWhere(checkField, ordinal + 1) != -1) {
320-
sys.error(s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}")
323+
throw new AnalysisException(
324+
s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}")
321325
} else {
322326
ordinal
323327
}
@@ -329,7 +333,8 @@ class Analyzer(catalog: Catalog,
329333
case ArrayType(StructType(fields), containsNull) =>
330334
val ordinal = findField(fields)
331335
ArrayGetField(expr, fields(ordinal), ordinal, containsNull)
332-
case otherType => sys.error(s"GetField is not valid on fields of type $otherType")
336+
case otherType =>
337+
throw new AnalysisException(s"GetField is not valid on fields of type $otherType")
333338
}
334339
}
335340
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.analysis
1919

2020
import org.scalatest.{BeforeAndAfter, FunSuite}
2121

22+
import org.apache.spark.sql.AnalysisException
2223
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference}
23-
import org.apache.spark.sql.catalyst.errors.TreeNodeException
2424
import org.apache.spark.sql.catalyst.plans.logical._
2525
import org.apache.spark.sql.types._
2626

@@ -69,12 +69,12 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
6969
UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) ===
7070
Project(testRelation.output, testRelation))
7171

72-
val e = intercept[TreeNodeException[_]] {
72+
val e = intercept[AnalysisException] {
7373
caseSensitiveAnalyze(
7474
Project(Seq(UnresolvedAttribute("tBl.a")),
7575
UnresolvedRelation(Seq("TaBlE"), Some("TbL"))))
7676
}
77-
assert(e.getMessage().toLowerCase.contains("unresolved"))
77+
assert(e.getMessage().toLowerCase.contains("cannot resolve"))
7878

7979
assert(
8080
caseInsensitiveAnalyze(
@@ -109,21 +109,21 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
109109
}
110110

111111
test("throw errors for unresolved attributes during analysis") {
112-
val e = intercept[TreeNodeException[_]] {
112+
val e = intercept[AnalysisException] {
113113
caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation))
114114
}
115-
assert(e.getMessage().toLowerCase.contains("unresolved attribute"))
115+
assert(e.getMessage().toLowerCase.contains("cannot resolve"))
116116
}
117117

118118
test("throw errors for unresolved plans during analysis") {
119119
case class UnresolvedTestPlan() extends LeafNode {
120120
override lazy val resolved = false
121121
override def output = Nil
122122
}
123-
val e = intercept[TreeNodeException[_]] {
123+
val e = intercept[AnalysisException] {
124124
caseSensitiveAnalyze(UnresolvedTestPlan())
125125
}
126-
assert(e.getMessage().toLowerCase.contains("unresolved plan"))
126+
assert(e.getMessage().toLowerCase.contains("unresolved"))
127127
}
128128

129129
test("divide should be casted into fractional types") {

sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -589,7 +589,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
589589
("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Row(_)))
590590
// Column type mismatches where a coercion is not possible, in this case between integer
591591
// and array types, trigger a TreeNodeException.
592-
intercept[TreeNodeException[_]] {
592+
intercept[AnalysisException] {
593593
sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect()
594594
}
595595
}

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.sql.hive.execution
1919

20+
import org.apache.spark.sql.AnalysisException
2021
import org.apache.spark.sql.hive.test.TestHive.{sparkContext, jsonRDD, sql}
2122
import org.apache.spark.sql.hive.test.TestHive.implicits._
2223

@@ -40,7 +41,7 @@ class HiveResolutionSuite extends HiveComparisonTest {
4041
"""{"a": [{"b": 1, "B": 2}]}""" :: Nil)).registerTempTable("nested")
4142

4243
// there are 2 filed matching field name "b", we should report Ambiguous reference error
43-
val exception = intercept[RuntimeException] {
44+
val exception = intercept[AnalysisException] {
4445
sql("SELECT a[0].b from nested").queryExecution.analyzed
4546
}
4647
assert(exception.getMessage.contains("Ambiguous reference to fields"))

0 commit comments

Comments
 (0)