From 79ee4a6603e48e8a755afadfe714d87f15dacac3 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 21 Apr 2017 17:48:08 +0200 Subject: [PATCH] Fix unit tests --- .../org/apache/spark/sql/AnalysisException.scala | 13 ++++++++++++- .../spark/sql/catalyst/analysis/Analyzer.scala | 13 ++++--------- .../sql-tests/results/change-column.sql.out | 4 ++-- .../test/resources/sql-tests/results/cte.sql.out | 4 ++-- .../resources/sql-tests/results/describe.sql.out | 2 +- .../sql-tests/results/show-tables.sql.out | 4 ++-- .../sql-tests/results/show_columns.sql.out | 8 ++++---- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../spark/sql/execution/SQLViewSuite.scala | 16 ++++++++-------- .../spark/sql/hive/thriftserver/CliSuite.scala | 2 +- 10 files changed, 37 insertions(+), 31 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 index d14dd6e1a25ee..ff8f9f3d82fd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -37,8 +37,19 @@ class AnalysisException( val cause: Option[Throwable] = None) extends Exception(message, cause.orNull) with Serializable { + def withPlan(plan: LogicalPlan): AnalysisException = { + withPosition(plan.origin.line, plan.origin.startPosition, Option(plan)) + } + def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { - val newException = new AnalysisException(message, line, startPosition) + withPosition(line, startPosition, None) + } + + private def withPosition( + line: Option[Int], + startPosition: Option[Int], + plan: Option[LogicalPlan]): AnalysisException = { + val newException = new AnalysisException(message, line, startPosition, plan) newException.setStackTrace(getStackTrace) newException } 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 1cc979fc26c45..4c8cb833ce2d1 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 @@ -636,16 +636,11 @@ class Analyzer( defaultDatabase: Option[String] = None): LogicalPlan = { val tableIdentWithDb = u.tableIdentifier.copy( database = u.tableIdentifier.database.orElse(defaultDatabase)) - if (!tableIdentWithDb.database.exists(catalog.databaseExists)) { - // Note that if the database is not defined, it is possible we are looking up a temp view. - u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}, the " + - s"database doesn't exsits.") + try { + catalog.lookupRelation(tableIdentWithDb) + } catch { + case a: AnalysisException => throw a.withPlan(u) } - if (!catalog.tableExists(tableIdentWithDb)) { - // If the database is defined and that database is not found, throw an AnalysisException. - u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}") - } - catalog.lookupRelation(tableIdentWithDb) } // If the database part is specified, and we support running SQL directly on files, and diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 678a3f0f0a3c6..0aaebbd1db009 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -256,7 +256,7 @@ ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a' -- !query 25 schema struct<> -- !query 25 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.AnalysisException Table or view 'temp_view' not found in database 'default'; @@ -273,7 +273,7 @@ ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column -- !query 27 schema struct<> -- !query 27 output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.AnalysisException Database 'global_temp' not found; diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index a446c2cd183da..baf60d32c9eaa 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -24,7 +24,7 @@ WITH s AS (SELECT 1 FROM s) SELECT * FROM s struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -Table or view not found: s; line 1 pos 25 +Table or view 's' not found in database 'default'; line 1 pos 25 -- !query 3 @@ -43,7 +43,7 @@ WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -Table or view not found: s2; line 1 pos 26 +Table or view 's2' not found in database 'default'; line 1 pos 26 -- !query 5 diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index de10b29f3c65b..de401e25e0d0d 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -234,7 +234,7 @@ DESC t PARTITION (c='Us', d=2) -- !query 13 schema struct<> -- !query 13 output -org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +org.apache.spark.sql.AnalysisException Partition not found in table 't' database 'default': c -> Us d -> 2; diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index 8f2a54f7c24e2..423b2edae858b 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -196,7 +196,7 @@ SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) -- !query 16 schema struct<> -- !query 16 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.AnalysisException Table or view 'show_t*' not found in database 'showdb'; @@ -223,7 +223,7 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) -- !query 19 schema struct<> -- !query 19 output -org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +org.apache.spark.sql.AnalysisException Partition not found in table 'show_t1' database 'showdb': c -> Ch d -> 1; diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out index 05c3a083ee3b3..92e01ac067a88 100644 --- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -93,7 +93,7 @@ SHOW COLUMNS IN badtable FROM showdb -- !query 10 schema struct<> -- !query 10 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.AnalysisException Table or view 'badtable' not found in database 'showdb'; @@ -129,7 +129,7 @@ SHOW COLUMNS IN showdb.showcolumn3 -- !query 14 schema struct<> -- !query 14 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.AnalysisException Table or view 'showcolumn3' not found in database 'showdb'; @@ -138,7 +138,7 @@ SHOW COLUMNS IN showcolumn3 FROM showdb -- !query 15 schema struct<> -- !query 15 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.AnalysisException Table or view 'showcolumn3' not found in database 'showdb'; @@ -147,7 +147,7 @@ SHOW COLUMNS IN showcolumn4 -- !query 16 schema struct<> -- !query 16 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.AnalysisException Table or view 'showcolumn4' not found in database 'showdb'; 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 0dd9296a3f0ff..3b3f8b7878120 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 @@ -1651,7 +1651,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { var e = intercept[AnalysisException] { sql("select * from in_valid_table") } - assert(e.message.contains("Table or view not found")) + assert(e.message.contains("Table or view 'in_valid_table' not found")) e = intercept[AnalysisException] { sql("select * from no_db.no_table").show() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index d05bca38f5607..a25b1f74df0e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -195,16 +195,16 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist345") // A column that does not exist - intercept[AnalysisException] { - sql("CREATE OR REPLACE VIEW myabcdview AS SELECT random1234 FROM jt").collect() - } + assertInvalid("CREATE OR REPLACE VIEW myabcdview AS SELECT random1234 FROM jt") } private def assertInvalidReference(query: String): Unit = { - val e = intercept[AnalysisException] { - sql(query) - }.getMessage - assert(e.contains("Table or view not found")) + assertInvalid(query, "Table or view", "not found") + } + + private def assertInvalid(query: String, msgs: String*): Unit = { + val e = intercept[AnalysisException](sql(query)).getMessage + msgs.foreach(msg => assert(e.contains(msg))) } @@ -528,7 +528,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } } - assertInvalidReference("SELECT * FROM view1") + assertInvalid("SELECT * FROM view1", "Database", "not found") // Fail if the referenced table is invalid. withTable("table2") { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index d3cec11bd7567..a09b3330b12c6 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -232,7 +232,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { runCliWithin(timeout = 2.minute, errorResponses = Seq("AnalysisException"))( "select * from nonexistent_table;" - -> "Error in query: Table or view not found: nonexistent_table;" + -> "Error in query: Table or view 'nonexistent_table' not found" ) }