Skip to content

Commit 79ee4a6

Browse files
committed
Fix unit tests
1 parent 6dc3204 commit 79ee4a6

File tree

10 files changed

+37
-31
lines changed

10 files changed

+37
-31
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,8 +37,19 @@ class AnalysisException(
3737
val cause: Option[Throwable] = None)
3838
extends Exception(message, cause.orNull) with Serializable {
3939

40+
def withPlan(plan: LogicalPlan): AnalysisException = {
41+
withPosition(plan.origin.line, plan.origin.startPosition, Option(plan))
42+
}
43+
4044
def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = {
41-
val newException = new AnalysisException(message, line, startPosition)
45+
withPosition(line, startPosition, None)
46+
}
47+
48+
private def withPosition(
49+
line: Option[Int],
50+
startPosition: Option[Int],
51+
plan: Option[LogicalPlan]): AnalysisException = {
52+
val newException = new AnalysisException(message, line, startPosition, plan)
4253
newException.setStackTrace(getStackTrace)
4354
newException
4455
}

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

Lines changed: 4 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -636,16 +636,11 @@ class Analyzer(
636636
defaultDatabase: Option[String] = None): LogicalPlan = {
637637
val tableIdentWithDb = u.tableIdentifier.copy(
638638
database = u.tableIdentifier.database.orElse(defaultDatabase))
639-
if (!tableIdentWithDb.database.exists(catalog.databaseExists)) {
640-
// Note that if the database is not defined, it is possible we are looking up a temp view.
641-
u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}, the " +
642-
s"database doesn't exsits.")
639+
try {
640+
catalog.lookupRelation(tableIdentWithDb)
641+
} catch {
642+
case a: AnalysisException => throw a.withPlan(u)
643643
}
644-
if (!catalog.tableExists(tableIdentWithDb)) {
645-
// If the database is defined and that database is not found, throw an AnalysisException.
646-
u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}")
647-
}
648-
catalog.lookupRelation(tableIdentWithDb)
649644
}
650645

651646
// If the database part is specified, and we support running SQL directly on files, and

sql/core/src/test/resources/sql-tests/results/change-column.sql.out

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -256,7 +256,7 @@ ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a'
256256
-- !query 25 schema
257257
struct<>
258258
-- !query 25 output
259-
org.apache.spark.sql.catalyst.analysis.NoSuchTableException
259+
org.apache.spark.sql.AnalysisException
260260
Table or view 'temp_view' not found in database 'default';
261261

262262

@@ -273,7 +273,7 @@ ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column
273273
-- !query 27 schema
274274
struct<>
275275
-- !query 27 output
276-
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
276+
org.apache.spark.sql.AnalysisException
277277
Database 'global_temp' not found;
278278

279279

sql/core/src/test/resources/sql-tests/results/cte.sql.out

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ WITH s AS (SELECT 1 FROM s) SELECT * FROM s
2424
struct<>
2525
-- !query 2 output
2626
org.apache.spark.sql.AnalysisException
27-
Table or view not found: s; line 1 pos 25
27+
Table or view 's' not found in database 'default'; line 1 pos 25
2828

2929

3030
-- !query 3
@@ -43,7 +43,7 @@ WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2
4343
struct<>
4444
-- !query 4 output
4545
org.apache.spark.sql.AnalysisException
46-
Table or view not found: s2; line 1 pos 26
46+
Table or view 's2' not found in database 'default'; line 1 pos 26
4747

4848

4949
-- !query 5

sql/core/src/test/resources/sql-tests/results/describe.sql.out

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -234,7 +234,7 @@ DESC t PARTITION (c='Us', d=2)
234234
-- !query 13 schema
235235
struct<>
236236
-- !query 13 output
237-
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
237+
org.apache.spark.sql.AnalysisException
238238
Partition not found in table 't' database 'default':
239239
c -> Us
240240
d -> 2;

sql/core/src/test/resources/sql-tests/results/show-tables.sql.out

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -196,7 +196,7 @@ SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1)
196196
-- !query 16 schema
197197
struct<>
198198
-- !query 16 output
199-
org.apache.spark.sql.catalyst.analysis.NoSuchTableException
199+
org.apache.spark.sql.AnalysisException
200200
Table or view 'show_t*' not found in database 'showdb';
201201

202202

@@ -223,7 +223,7 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1)
223223
-- !query 19 schema
224224
struct<>
225225
-- !query 19 output
226-
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
226+
org.apache.spark.sql.AnalysisException
227227
Partition not found in table 'show_t1' database 'showdb':
228228
c -> Ch
229229
d -> 1;

sql/core/src/test/resources/sql-tests/results/show_columns.sql.out

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,7 @@ SHOW COLUMNS IN badtable FROM showdb
9393
-- !query 10 schema
9494
struct<>
9595
-- !query 10 output
96-
org.apache.spark.sql.catalyst.analysis.NoSuchTableException
96+
org.apache.spark.sql.AnalysisException
9797
Table or view 'badtable' not found in database 'showdb';
9898

9999

@@ -129,7 +129,7 @@ SHOW COLUMNS IN showdb.showcolumn3
129129
-- !query 14 schema
130130
struct<>
131131
-- !query 14 output
132-
org.apache.spark.sql.catalyst.analysis.NoSuchTableException
132+
org.apache.spark.sql.AnalysisException
133133
Table or view 'showcolumn3' not found in database 'showdb';
134134

135135

@@ -138,7 +138,7 @@ SHOW COLUMNS IN showcolumn3 FROM showdb
138138
-- !query 15 schema
139139
struct<>
140140
-- !query 15 output
141-
org.apache.spark.sql.catalyst.analysis.NoSuchTableException
141+
org.apache.spark.sql.AnalysisException
142142
Table or view 'showcolumn3' not found in database 'showdb';
143143

144144

@@ -147,7 +147,7 @@ SHOW COLUMNS IN showcolumn4
147147
-- !query 16 schema
148148
struct<>
149149
-- !query 16 output
150-
org.apache.spark.sql.catalyst.analysis.NoSuchTableException
150+
org.apache.spark.sql.AnalysisException
151151
Table or view 'showcolumn4' not found in database 'showdb';
152152

153153

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
@@ -1651,7 +1651,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
16511651
var e = intercept[AnalysisException] {
16521652
sql("select * from in_valid_table")
16531653
}
1654-
assert(e.message.contains("Table or view not found"))
1654+
assert(e.message.contains("Table or view 'in_valid_table' not found"))
16551655

16561656
e = intercept[AnalysisException] {
16571657
sql("select * from no_db.no_table").show()

sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -195,16 +195,16 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
195195
assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist345")
196196

197197
// A column that does not exist
198-
intercept[AnalysisException] {
199-
sql("CREATE OR REPLACE VIEW myabcdview AS SELECT random1234 FROM jt").collect()
200-
}
198+
assertInvalid("CREATE OR REPLACE VIEW myabcdview AS SELECT random1234 FROM jt")
201199
}
202200

203201
private def assertInvalidReference(query: String): Unit = {
204-
val e = intercept[AnalysisException] {
205-
sql(query)
206-
}.getMessage
207-
assert(e.contains("Table or view not found"))
202+
assertInvalid(query, "Table or view", "not found")
203+
}
204+
205+
private def assertInvalid(query: String, msgs: String*): Unit = {
206+
val e = intercept[AnalysisException](sql(query)).getMessage
207+
msgs.foreach(msg => assert(e.contains(msg)))
208208
}
209209

210210

@@ -528,7 +528,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
528528
}
529529
}
530530
}
531-
assertInvalidReference("SELECT * FROM view1")
531+
assertInvalid("SELECT * FROM view1", "Database", "not found")
532532

533533
// Fail if the referenced table is invalid.
534534
withTable("table2") {

sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -232,7 +232,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
232232
runCliWithin(timeout = 2.minute,
233233
errorResponses = Seq("AnalysisException"))(
234234
"select * from nonexistent_table;"
235-
-> "Error in query: Table or view not found: nonexistent_table;"
235+
-> "Error in query: Table or view 'nonexistent_table' not found"
236236
)
237237
}
238238

0 commit comments

Comments
 (0)