Skip to content

Commit 60f3a73

Browse files
imback82cloud-fan
authored andcommitted
[SPARK-33515][SQL] Improve exception messages while handling UnresolvedTable
### What changes were proposed in this pull request? This PR proposes to improve the exception messages while `UnresolvedTable` is handled based on this suggestion: #30321 (comment). Currently, when an identifier is resolved to a view when a table is expected, the following exception message is displayed (e.g., for `COMMENT ON TABLE`): ``` v is a temp view not table. ``` After this PR, the message will be: ``` v is a temp view. 'COMMENT ON TABLE' expects a table. ``` Also, if an identifier is not resolved, the following exception message is currently used: ``` Table not found: t ``` After this PR, the message will be: ``` Table not found for 'COMMENT ON TABLE': t ``` ### Why are the changes needed? To improve the exception message. ### Does this PR introduce _any_ user-facing change? Yes, the exception message will be changed as described above. ### How was this patch tested? Updated existing tests. Closes #30461 from imback82/unresolved_table_message. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent c891e02 commit 60f3a73

File tree

8 files changed

+34
-27
lines changed

8 files changed

+34
-27
lines changed

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -861,9 +861,9 @@ class Analyzer(override val catalogManager: CatalogManager)
861861
}.getOrElse(write)
862862
case _ => write
863863
}
864-
case u @ UnresolvedTable(ident) =>
864+
case u @ UnresolvedTable(ident, cmd) =>
865865
lookupTempView(ident).foreach { _ =>
866-
u.failAnalysis(s"${ident.quoted} is a temp view not table.")
866+
u.failAnalysis(s"${ident.quoted} is a temp view. '$cmd' expects a table")
867867
}
868868
u
869869
case u @ UnresolvedTableOrView(ident, allowTempView) =>
@@ -950,7 +950,7 @@ class Analyzer(override val catalogManager: CatalogManager)
950950
SubqueryAlias(catalog.get.name +: ident.namespace :+ ident.name, relation)
951951
}.getOrElse(u)
952952

953-
case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident)) =>
953+
case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident), _) =>
954954
CatalogV2Util.loadTable(catalog, ident)
955955
.map(ResolvedTable(catalog.asTableCatalog, ident, _))
956956
.getOrElse(u)
@@ -1077,11 +1077,11 @@ class Analyzer(override val catalogManager: CatalogManager)
10771077
lookupRelation(u.multipartIdentifier, u.options, u.isStreaming)
10781078
.map(resolveViews).getOrElse(u)
10791079

1080-
case u @ UnresolvedTable(identifier) =>
1080+
case u @ UnresolvedTable(identifier, cmd) =>
10811081
lookupTableOrView(identifier).map {
10821082
case v: ResolvedView =>
10831083
val viewStr = if (v.isTemp) "temp view" else "view"
1084-
u.failAnalysis(s"${v.identifier.quoted} is a $viewStr not table.")
1084+
u.failAnalysis(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.'")
10851085
case table => table
10861086
}.getOrElse(u)
10871087

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -98,7 +98,7 @@ trait CheckAnalysis extends PredicateHelper {
9898
u.failAnalysis(s"Namespace not found: ${u.multipartIdentifier.quoted}")
9999

100100
case u: UnresolvedTable =>
101-
u.failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}")
101+
u.failAnalysis(s"Table not found for '${u.commandName}': ${u.multipartIdentifier.quoted}")
102102

103103
case u: UnresolvedTableOrView =>
104104
u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}")

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,9 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod
3737
* Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
3838
* [[ResolvedTable]] during analysis.
3939
*/
40-
case class UnresolvedTable(multipartIdentifier: Seq[String]) extends LeafNode {
40+
case class UnresolvedTable(
41+
multipartIdentifier: Seq[String],
42+
commandName: String) extends LeafNode {
4143
override lazy val resolved: Boolean = false
4244

4345
override def output: Seq[Attribute] = Nil

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -3303,7 +3303,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
33033303
*/
33043304
override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) {
33053305
LoadData(
3306-
child = UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)),
3306+
child = UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier), "LOAD DATA"),
33073307
path = string(ctx.path),
33083308
isLocal = ctx.LOCAL != null,
33093309
isOverwrite = ctx.OVERWRITE != null,
@@ -3449,7 +3449,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
34493449
UnresolvedPartitionSpec(spec, location)
34503450
}
34513451
AlterTableAddPartition(
3452-
UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)),
3452+
UnresolvedTable(
3453+
visitMultipartIdentifier(ctx.multipartIdentifier),
3454+
"ALTER TABLE ... ADD PARTITION ..."),
34533455
specsAndLocs.toSeq,
34543456
ctx.EXISTS != null)
34553457
}
@@ -3491,7 +3493,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
34913493
val partSpecs = ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec)
34923494
.map(spec => UnresolvedPartitionSpec(spec))
34933495
AlterTableDropPartition(
3494-
UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)),
3496+
UnresolvedTable(
3497+
visitMultipartIdentifier(ctx.multipartIdentifier),
3498+
"ALTER TABLE ... DROP PARTITION ..."),
34953499
partSpecs.toSeq,
34963500
ifExists = ctx.EXISTS != null,
34973501
purge = ctx.PURGE != null,
@@ -3720,6 +3724,6 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
37203724
case _ => string(ctx.STRING)
37213725
}
37223726
val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
3723-
CommentOnTable(UnresolvedTable(nameParts), comment)
3727+
CommentOnTable(UnresolvedTable(nameParts, "COMMENT ON TABLE"), comment)
37243728
}
37253729
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1555,15 +1555,15 @@ class DDLParserSuite extends AnalysisTest {
15551555
test("LOAD DATA INTO table") {
15561556
comparePlans(
15571557
parsePlan("LOAD DATA INPATH 'filepath' INTO TABLE a.b.c"),
1558-
LoadData(UnresolvedTable(Seq("a", "b", "c")), "filepath", false, false, None))
1558+
LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", false, false, None))
15591559

15601560
comparePlans(
15611561
parsePlan("LOAD DATA LOCAL INPATH 'filepath' INTO TABLE a.b.c"),
1562-
LoadData(UnresolvedTable(Seq("a", "b", "c")), "filepath", true, false, None))
1562+
LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", true, false, None))
15631563

15641564
comparePlans(
15651565
parsePlan("LOAD DATA LOCAL INPATH 'filepath' OVERWRITE INTO TABLE a.b.c"),
1566-
LoadData(UnresolvedTable(Seq("a", "b", "c")), "filepath", true, true, None))
1566+
LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", true, true, None))
15671567

15681568
comparePlans(
15691569
parsePlan(
@@ -1572,7 +1572,7 @@ class DDLParserSuite extends AnalysisTest {
15721572
|PARTITION(ds='2017-06-10')
15731573
""".stripMargin),
15741574
LoadData(
1575-
UnresolvedTable(Seq("a", "b", "c")),
1575+
UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"),
15761576
"filepath",
15771577
true,
15781578
true,
@@ -1674,13 +1674,13 @@ class DDLParserSuite extends AnalysisTest {
16741674
val parsed2 = parsePlan(sql2)
16751675

16761676
val expected1 = AlterTableAddPartition(
1677-
UnresolvedTable(Seq("a", "b", "c")),
1677+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ..."),
16781678
Seq(
16791679
UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")),
16801680
UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"), None)),
16811681
ifNotExists = true)
16821682
val expected2 = AlterTableAddPartition(
1683-
UnresolvedTable(Seq("a", "b", "c")),
1683+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ..."),
16841684
Seq(UnresolvedPartitionSpec(Map("dt" -> "2008-08-08"), Some("loc"))),
16851685
ifNotExists = false)
16861686

@@ -1747,7 +1747,7 @@ class DDLParserSuite extends AnalysisTest {
17471747
assertUnsupported(sql2_view)
17481748

17491749
val expected1_table = AlterTableDropPartition(
1750-
UnresolvedTable(Seq("table_name")),
1750+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP PARTITION ..."),
17511751
Seq(
17521752
UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
17531753
UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))),
@@ -1763,7 +1763,7 @@ class DDLParserSuite extends AnalysisTest {
17631763

17641764
val sql3_table = "ALTER TABLE a.b.c DROP IF EXISTS PARTITION (ds='2017-06-10')"
17651765
val expected3_table = AlterTableDropPartition(
1766-
UnresolvedTable(Seq("a", "b", "c")),
1766+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... DROP PARTITION ..."),
17671767
Seq(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10"))),
17681768
ifExists = true,
17691769
purge = false,
@@ -2174,7 +2174,7 @@ class DDLParserSuite extends AnalysisTest {
21742174

21752175
comparePlans(
21762176
parsePlan("COMMENT ON TABLE a.b.c IS 'xYz'"),
2177-
CommentOnTable(UnresolvedTable(Seq("a", "b", "c")), "xYz"))
2177+
CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE"), "xYz"))
21782178
}
21792179

21802180
// TODO: ignored by SPARK-31707, restore the test after create table syntax unification

sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2414,7 +2414,8 @@ class DataSourceV2SQLSuite
24142414
withTempView("v") {
24152415
sql("create global temp view v as select 1")
24162416
val e = intercept[AnalysisException](sql("COMMENT ON TABLE global_temp.v IS NULL"))
2417-
assert(e.getMessage.contains("global_temp.v is a temp view not table."))
2417+
assert(e.getMessage.contains(
2418+
"global_temp.v is a temp view. 'COMMENT ON TABLE' expects a table"))
24182419
}
24192420
}
24202421

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

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -147,10 +147,10 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
147147
s"'$viewName' is a view not a table")
148148
assertAnalysisError(
149149
s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')",
150-
s"$viewName is a temp view not table")
150+
s"$viewName is a temp view. 'ALTER TABLE ... ADD PARTITION ...' expects a table")
151151
assertAnalysisError(
152152
s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')",
153-
s"$viewName is a temp view not table")
153+
s"$viewName is a temp view. 'ALTER TABLE ... DROP PARTITION ...' expects a table")
154154

155155
// For the following v2 ALERT TABLE statements, unsupported operations are checked first
156156
// before resolving the relations.
@@ -175,7 +175,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
175175
val e2 = intercept[AnalysisException] {
176176
sql(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""")
177177
}.getMessage
178-
assert(e2.contains(s"$viewName is a temp view not table"))
178+
assert(e2.contains(s"$viewName is a temp view. 'LOAD DATA' expects a table"))
179179
assertNoSuchTable(s"TRUNCATE TABLE $viewName")
180180
val e3 = intercept[AnalysisException] {
181181
sql(s"SHOW CREATE TABLE $viewName")
@@ -214,7 +214,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
214214
e = intercept[AnalysisException] {
215215
sql(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""")
216216
}.getMessage
217-
assert(e.contains("default.testView is a view not table"))
217+
assert(e.contains("default.testView is a view. 'LOAD DATA' expects a table"))
218218

219219
e = intercept[AnalysisException] {
220220
sql(s"TRUNCATE TABLE $viewName")

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -904,10 +904,10 @@ class HiveDDLSuite
904904

905905
assertAnalysisError(
906906
s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')",
907-
s"$oldViewName is a view not table")
907+
s"$oldViewName is a view. 'ALTER TABLE ... ADD PARTITION ...' expects a table.")
908908
assertAnalysisError(
909909
s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')",
910-
s"$oldViewName is a view not table")
910+
s"$oldViewName is a view. 'ALTER TABLE ... DROP PARTITION ...' expects a table.")
911911

912912
assert(catalog.tableExists(TableIdentifier(tabName)))
913913
assert(catalog.tableExists(TableIdentifier(oldViewName)))

0 commit comments

Comments
 (0)