Skip to content

Commit

Permalink
Fix unit tests
Browse files Browse the repository at this point in the history
  • Loading branch information
hvanhovell committed Apr 21, 2017
1 parent 6dc3204 commit 79ee4a6
Show file tree
Hide file tree
Showing 10 changed files with 37 additions and 31 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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';


Expand All @@ -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;


Expand Down
4 changes: 2 additions & 2 deletions sql/core/src/test/resources/sql-tests/results/cte.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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';


Expand All @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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';


Expand Down Expand Up @@ -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';


Expand All @@ -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';


Expand All @@ -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';


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)))
}


Expand Down Expand Up @@ -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") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
}

Expand Down

0 comments on commit 79ee4a6

Please sign in to comment.