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 837686420375..77c1dd9ebb7f 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 @@ -866,11 +866,12 @@ class Analyzer(override val catalogManager: CatalogManager) u.failAnalysis(s"${ident.quoted} is a temp view. '$cmd' expects a table") } u - case u @ UnresolvedTableOrView(ident, allowTempView) => + case u @ UnresolvedTableOrView(ident, cmd, allowTempView) => lookupTempView(ident) .map { _ => if (!allowTempView) { - u.failAnalysis(s"${ident.quoted} is a temp view not table or permanent view.") + u.failAnalysis( + s"${ident.quoted} is a temp view. '$cmd' expects a table or permanent view.") } ResolvedView(ident.asIdentifier, isTemp = true) } @@ -955,7 +956,7 @@ class Analyzer(override val catalogManager: CatalogManager) .map(ResolvedTable(catalog.asTableCatalog, ident, _)) .getOrElse(u) - case u @ UnresolvedTableOrView(NonSessionCatalogAndIdentifier(catalog, ident), _) => + case u @ UnresolvedTableOrView(NonSessionCatalogAndIdentifier(catalog, ident), _, _) => CatalogV2Util.loadTable(catalog, ident) .map(ResolvedTable(catalog.asTableCatalog, ident, _)) .getOrElse(u) @@ -1085,7 +1086,7 @@ class Analyzer(override val catalogManager: CatalogManager) case table => table }.getOrElse(u) - case u @ UnresolvedTableOrView(identifier, _) => + case u @ UnresolvedTableOrView(identifier, _, _) => lookupTableOrView(identifier).getOrElse(u) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 9998035d65c3..9a3ab4a5f8d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -101,7 +101,9 @@ trait CheckAnalysis extends PredicateHelper { u.failAnalysis(s"Table not found for '${u.commandName}': ${u.multipartIdentifier.quoted}") case u: UnresolvedTableOrView => - u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") + val viewStr = if (u.allowTempView) "view" else "permanent view" + u.failAnalysis( + s"Table or $viewStr not found for '${u.commandName}': ${u.multipartIdentifier.quoted}") case u: UnresolvedRelation => u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 0e883a88f269..95fc4f47dec7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -51,6 +51,7 @@ case class UnresolvedTable( */ case class UnresolvedTableOrView( multipartIdentifier: Seq[String], + commandName: String, allowTempView: Boolean = true) extends LeafNode { override lazy val resolved: Boolean = false override def output: Seq[Attribute] = Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 606d92306144..4cd9b2bea32a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3148,7 +3148,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { // DROP TABLE works with either a table or a temporary view. DropTable( - UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())), + UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier()), "DROP TABLE"), ctx.EXISTS != null, ctx.PURGE != null) } @@ -3453,12 +3453,15 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null + val relation = UnresolvedTableOrView( + visitMultipartIdentifier(ctx.multipartIdentifier()), + "DESCRIBE TABLE") if (ctx.describeColName != null) { if (ctx.partitionSpec != null) { throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) } else { DescribeColumn( - UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())), + relation, ctx.describeColName.nameParts.asScala.map(_.getText).toSeq, isExtended) } @@ -3473,10 +3476,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } else { Map.empty[String, String] } - DescribeRelation( - UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())), - partitionSpec, - isExtended) + DescribeRelation(relation, partitionSpec, isExtended) } } @@ -3514,7 +3514,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val tableName = visitMultipartIdentifier(ctx.multipartIdentifier()) if (ctx.ALL() != null) { checkPartitionSpec() - AnalyzeColumn(UnresolvedTableOrView(tableName), None, allColumns = true) + AnalyzeColumn( + UnresolvedTableOrView(tableName, "ANALYZE TABLE ... FOR ALL COLUMNS"), + None, + allColumns = true) } else if (ctx.identifierSeq() == null) { val partitionSpec = if (ctx.partitionSpec != null) { visitPartitionSpec(ctx.partitionSpec) @@ -3522,13 +3525,13 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg Map.empty[String, Option[String]] } AnalyzeTable( - UnresolvedTableOrView(tableName, allowTempView = false), + UnresolvedTableOrView(tableName, "ANALYZE TABLE", allowTempView = false), partitionSpec, noScan = ctx.identifier != null) } else { checkPartitionSpec() AnalyzeColumn( - UnresolvedTableOrView(tableName), + UnresolvedTableOrView(tableName, "ANALYZE TABLE ... FOR COLUMNS ..."), Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) } @@ -3572,6 +3575,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg ShowCreateTable( UnresolvedTableOrView( visitMultipartIdentifier(ctx.multipartIdentifier()), + "SHOW CREATE TABLE", allowTempView = false), ctx.SERDE != null) } @@ -3647,7 +3651,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg * }}} */ override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { - RefreshTable(UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier()))) + RefreshTable( + UnresolvedTableOrView( + visitMultipartIdentifier(ctx.multipartIdentifier()), + "REFRESH TABLE")) } /** @@ -3670,7 +3677,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } else { nameParts } - ShowColumns(UnresolvedTableOrView(tableName), namespace) + ShowColumns(UnresolvedTableOrView(tableName, "SHOW COLUMNS"), namespace) } /** @@ -3881,7 +3888,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg override def visitShowTblProperties( ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) { ShowTableProperties( - UnresolvedTableOrView(visitMultipartIdentifier(ctx.table)), + UnresolvedTableOrView(visitMultipartIdentifier(ctx.table), "SHOW TBLPROPERTIES"), Option(ctx.key).map(visitTablePropertyKey)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index f650922e75f6..c58ff81f1713 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -697,27 +697,27 @@ class DDLParserSuite extends AnalysisTest { test("drop table") { parseCompare("DROP TABLE testcat.ns1.ns2.tbl", DropTable( - UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl")), + UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl"), "DROP TABLE"), ifExists = false, purge = false)) parseCompare(s"DROP TABLE db.tab", DropTable( - UnresolvedTableOrView(Seq("db", "tab")), ifExists = false, purge = false)) + UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE"), ifExists = false, purge = false)) parseCompare(s"DROP TABLE IF EXISTS db.tab", DropTable( - UnresolvedTableOrView(Seq("db", "tab")), ifExists = true, purge = false)) + UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE"), ifExists = true, purge = false)) parseCompare(s"DROP TABLE tab", DropTable( - UnresolvedTableOrView(Seq("tab")), ifExists = false, purge = false)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = false, purge = false)) parseCompare(s"DROP TABLE IF EXISTS tab", DropTable( - UnresolvedTableOrView(Seq("tab")), ifExists = true, purge = false)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = true, purge = false)) parseCompare(s"DROP TABLE tab PURGE", DropTable( - UnresolvedTableOrView(Seq("tab")), ifExists = false, purge = true)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = false, purge = true)) parseCompare(s"DROP TABLE IF EXISTS tab PURGE", DropTable( - UnresolvedTableOrView(Seq("tab")), ifExists = true, purge = true)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = true, purge = true)) } test("drop view") { @@ -1112,26 +1112,26 @@ class DDLParserSuite extends AnalysisTest { test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = false)) comparePlans(parsePlan("DESCRIBE t `abc.xyz`"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("abc.xyz"), isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("abc.xyz"), isExtended = false)) comparePlans(parsePlan("DESCRIBE t abc.xyz"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("abc", "xyz"), isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("abc", "xyz"), isExtended = false)) comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("a.b", "x.y"), isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("a.b", "x.y"), isExtended = false)) comparePlans(parsePlan("DESCRIBE TABLE t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = false)) comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = true)) comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = true)) val caught = intercept[AnalysisException]( parsePlan("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col")) @@ -1150,13 +1150,17 @@ class DDLParserSuite extends AnalysisTest { test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), - DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = false)) + DescribeRelation( + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = false)) comparePlans(parsePlan("describe table t"), - DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = false)) + DescribeRelation( + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = false)) comparePlans(parsePlan("describe table extended t"), - DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = true)) + DescribeRelation( + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = true)) comparePlans(parsePlan("describe table formatted t"), - DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = true)) + DescribeRelation( + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = true)) } test("insert table: basic append") { @@ -1769,57 +1773,57 @@ class DDLParserSuite extends AnalysisTest { test("analyze table statistics") { comparePlans(parsePlan("analyze table a.b.c compute statistics"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map.empty, noScan = false)) comparePlans(parsePlan("analyze table a.b.c compute statistics noscan"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map.empty, noScan = true)) comparePlans(parsePlan("analyze table a.b.c partition (a) compute statistics nOscAn"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("a" -> None), noScan = true)) // Partitions specified comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = false)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = true)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> Some("2008-04-09")), noScan = true)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = false)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = true)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr=11) COMPUTE STATISTICS noscan"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> None, "hr" -> Some("11")), noScan = true)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> None, "hr" -> None), noScan = false)) comparePlans( parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS noscan"), AnalyzeTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> None, "hr" -> None), noScan = true)) intercept("analyze table a.b.c compute statistics xxxx", @@ -1834,7 +1838,9 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS key, value"), AnalyzeColumn( - UnresolvedTableOrView(Seq("a", "b", "c")), Option(Seq("key", "value")), allColumns = false)) + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ..."), + Option(Seq("key", "value")), + allColumns = false)) // Partition specified - should be ignored comparePlans( @@ -1844,7 +1850,9 @@ class DDLParserSuite extends AnalysisTest { |COMPUTE STATISTICS FOR COLUMNS key, value """.stripMargin), AnalyzeColumn( - UnresolvedTableOrView(Seq("a", "b", "c")), Option(Seq("key", "value")), allColumns = false)) + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ..."), + Option(Seq("key", "value")), + allColumns = false)) // Partition specified should be ignored in case of COMPUTE STATISTICS FOR ALL COLUMNS comparePlans( @@ -1854,7 +1862,9 @@ class DDLParserSuite extends AnalysisTest { |COMPUTE STATISTICS FOR ALL COLUMNS """.stripMargin), AnalyzeColumn( - UnresolvedTableOrView(Seq("a", "b", "c")), None, allColumns = true)) + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR ALL COLUMNS"), + None, + allColumns = true)) intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value", "mismatched input 'key' expecting {, ';'}") @@ -1898,12 +1908,13 @@ class DDLParserSuite extends AnalysisTest { test("SHOW CREATE table") { comparePlans( parsePlan("SHOW CREATE TABLE a.b.c"), - ShowCreateTable(UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false))) + ShowCreateTable( + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW CREATE TABLE", allowTempView = false))) comparePlans( parsePlan("SHOW CREATE TABLE a.b.c AS SERDE"), ShowCreateTable( - UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false), + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW CREATE TABLE", allowTempView = false), asSerde = true)) } @@ -1949,7 +1960,7 @@ class DDLParserSuite extends AnalysisTest { test("REFRESH TABLE") { comparePlans( parsePlan("REFRESH TABLE a.b.c"), - RefreshTable(UnresolvedTableOrView(Seq("a", "b", "c")))) + RefreshTable(UnresolvedTableOrView(Seq("a", "b", "c"), "REFRESH TABLE"))) } test("show columns") { @@ -1959,13 +1970,15 @@ class DDLParserSuite extends AnalysisTest { val sql4 = "SHOW COLUMNS FROM db1.t1 IN db1" val parsed1 = parsePlan(sql1) - val expected1 = ShowColumns(UnresolvedTableOrView(Seq("t1")), None) + val expected1 = ShowColumns(UnresolvedTableOrView(Seq("t1"), "SHOW COLUMNS"), None) val parsed2 = parsePlan(sql2) - val expected2 = ShowColumns(UnresolvedTableOrView(Seq("db1", "t1")), None) + val expected2 = ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS"), None) val parsed3 = parsePlan(sql3) - val expected3 = ShowColumns(UnresolvedTableOrView(Seq("db1", "t1")), Some(Seq("db1"))) + val expected3 = + ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS"), Some(Seq("db1"))) val parsed4 = parsePlan(sql4) - val expected4 = ShowColumns(UnresolvedTableOrView(Seq("db1", "t1")), Some(Seq("db1"))) + val expected4 = + ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS"), Some(Seq("db1"))) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -2300,11 +2313,12 @@ class DDLParserSuite extends AnalysisTest { test("SHOW TBLPROPERTIES table") { comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c"), - ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), None)) + ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES"), None)) comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"), - ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), Some("propKey1"))) + ShowTableProperties( + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES"), Some("propKey1"))) } test("DESCRIBE FUNCTION") { 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 07aed98d120f..145c987ee5f6 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 @@ -540,7 +540,7 @@ struct -- !query output == Parsed Logical Plan == 'DescribeRelation false -+- 'UnresolvedTableOrView [t], true ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true == Analyzed Logical Plan == col_name: string, data_type: string, comment: string 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 6ddffb89987d..03df876133aa 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 @@ -94,7 +94,7 @@ SHOW COLUMNS IN badtable FROM showdb struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.badtable; line 1 pos 0 +Table or view not found for 'SHOW COLUMNS': showdb.badtable; line 1 pos 0 -- !query @@ -130,7 +130,7 @@ SHOW COLUMNS IN showdb.showcolumn3 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.showcolumn3; line 1 pos 0 +Table or view not found for 'SHOW COLUMNS': showdb.showcolumn3; line 1 pos 0 -- !query @@ -139,7 +139,7 @@ SHOW COLUMNS IN showcolumn3 FROM showdb struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.showcolumn3; line 1 pos 0 +Table or view not found for 'SHOW COLUMNS': showdb.showcolumn3; line 1 pos 0 -- !query @@ -148,7 +148,7 @@ SHOW COLUMNS IN showcolumn4 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showcolumn4; line 1 pos 0 +Table or view not found for 'SHOW COLUMNS': showcolumn4; line 1 pos 0 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index 7b4c8d1cc71d..92d306c0e3c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -155,7 +155,8 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { val ex = intercept[AnalysisException] { sql(s"SHOW CREATE TABLE $viewName") } - assert(ex.getMessage.contains(s"$viewName is a temp view not table or permanent view")) + assert(ex.getMessage.contains( + s"$viewName is a temp view. 'SHOW CREATE TABLE' expects a table or permanent view.")) } withGlobalTempView(viewName) { @@ -165,7 +166,8 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { sql(s"SHOW CREATE TABLE $globalTempViewDb.$viewName") } assert(ex.getMessage.contains( - s"$globalTempViewDb.$viewName is a temp view not table or permanent view")) + s"$globalTempViewDb.$viewName is a temp view. " + + "'SHOW CREATE TABLE' expects a table or permanent view.")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index cd03fadf34b9..3fc679f6b9fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -542,7 +542,8 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared val errMsg1 = intercept[AnalysisException] { sql(s"ANALYZE TABLE $globalTempDB.gTempView COMPUTE STATISTICS FOR COLUMNS id") }.getMessage - assert(errMsg1.contains(s"Table or view not found: $globalTempDB.gTempView")) + assert(errMsg1.contains("Table or view not found for 'ANALYZE TABLE ... FOR COLUMNS ...': " + + s"$globalTempDB.gTempView")) // Analyzes in a global temporary view sql("CREATE GLOBAL TEMP VIEW gTempView AS SELECT * FROM range(1, 30)") val errMsg2 = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index f2b57f9442d0..98580568a8df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -729,7 +729,7 @@ class DataSourceV2SQLSuite val ex = intercept[AnalysisException] { sql("DROP TABLE testcat.db.notbl") } - assert(ex.getMessage.contains("Table or view not found: testcat.db.notbl")) + assert(ex.getMessage.contains("Table or view not found for 'DROP TABLE': testcat.db.notbl")) sql("DROP TABLE IF EXISTS testcat.db.notbl") } 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 5d2950384877..d776198bc747 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 @@ -183,11 +183,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { val e4 = intercept[AnalysisException] { sql(s"SHOW CREATE TABLE $viewName") }.getMessage - assert(e4.contains(s"$viewName is a temp view not table or permanent view")) + assert(e4.contains( + s"$viewName is a temp view. 'SHOW CREATE TABLE' expects a table or permanent view.")) val e5 = intercept[AnalysisException] { sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") }.getMessage - assert(e5.contains(s"$viewName is a temp view not table or permanent view")) + assert(e5.contains( + s"$viewName is a temp view. 'ANALYZE TABLE' expects a table or permanent view.")) val e6 = intercept[AnalysisException] { sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") }.getMessage diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index c7ad96c8f761..97dd92acc780 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -80,8 +80,10 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("DROP TABLE h2.test.to_drop") checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) Seq( - "h2.test.not_existing_table" -> "Table or view not found: h2.test.not_existing_table", - "h2.bad_test.not_existing_table" -> "Table or view not found: h2.bad_test.not_existing_table" + "h2.test.not_existing_table" -> + "Table or view not found for 'DROP TABLE': h2.test.not_existing_table", + "h2.bad_test.not_existing_table" -> + "Table or view not found for 'DROP TABLE': h2.bad_test.not_existing_table" ).foreach { case (table, expectedMsg) => val msg = intercept[AnalysisException] { sql(s"DROP TABLE $table") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index d3398842afb2..4feb970ea6f1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -137,7 +137,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES badtable") }.getMessage - assert(message.contains("Table or view not found: badtable")) + assert(message.contains("Table or view not found for 'SHOW TBLPROPERTIES': badtable")) // When key is not found, a row containing the error is returned. checkAnswer(