Skip to content

Commit f024d30

Browse files
imback82cloud-fan
authored andcommitted
[SPARK-34317][SQL] Introduce relationTypeMismatchHint to UnresolvedTable for a better error message
### What changes were proposed in this pull request? This PR proposes to add `relationTypeMismatchHint` to `UnresolvedTable` so that if a relation is resolved to a view when a table is expected, a hint message can be included as a part of the analysis exception message. Note that the same feature is already introduced to `UnresolvedView` in #30636. This mostly affects `ALTER TABLE` commands where the analysis exception message will now contain `Please use ALTER VIEW as instead`. ### Why are the changes needed? To give a better error message. (The hint used to exist but got removed for commands that migrated to the new resolution framework) ### Does this PR introduce _any_ user-facing change? Yes, now `ALTER TABLE` commands include a hint to use `ALTER VIEW` instead. ``` sql("ALTER TABLE v SET SERDE 'whatever'") ``` Before: ``` "v is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table. ``` After this PR: ``` "v is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table. Please use ALTER VIEW instead. ``` ### How was this patch tested? Updated existing test cases to include the hint. Closes #31424 from imback82/better_error. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent bb9bf66 commit f024d30

File tree

12 files changed

+177
-79
lines changed

12 files changed

+177
-79
lines changed

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

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -893,7 +893,7 @@ class Analyzer(override val catalogManager: CatalogManager)
893893
}.getOrElse(write)
894894
case _ => write
895895
}
896-
case u @ UnresolvedTable(ident, cmd) =>
896+
case u @ UnresolvedTable(ident, cmd, _) =>
897897
lookupTempView(ident).foreach { _ =>
898898
throw QueryCompilationErrors.expectTableNotTempViewError(ident.quoted, cmd, u)
899899
}
@@ -990,7 +990,7 @@ class Analyzer(override val catalogManager: CatalogManager)
990990
SubqueryAlias(catalog.get.name +: ident.namespace :+ ident.name, relation)
991991
}.getOrElse(u)
992992

993-
case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident), _) =>
993+
case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident), _, _) =>
994994
CatalogV2Util.loadTable(catalog, ident)
995995
.map(table => ResolvedTable.create(catalog.asTableCatalog, ident, table))
996996
.getOrElse(u)
@@ -1144,18 +1144,20 @@ class Analyzer(override val catalogManager: CatalogManager)
11441144
lookupRelation(u.multipartIdentifier, u.options, u.isStreaming)
11451145
.map(resolveViews).getOrElse(u)
11461146

1147-
case u @ UnresolvedTable(identifier, cmd) =>
1147+
case u @ UnresolvedTable(identifier, cmd, relationTypeMismatchHint) =>
11481148
lookupTableOrView(identifier).map {
1149-
case v: ResolvedView => throw QueryCompilationErrors.expectTableNotViewError(v, cmd, u)
1149+
case v: ResolvedView =>
1150+
throw QueryCompilationErrors.expectTableNotViewError(
1151+
v, cmd, relationTypeMismatchHint, u)
11501152
case table => table
11511153
}.getOrElse(u)
11521154

11531155
case u @ UnresolvedView(identifier, cmd, _, relationTypeMismatchHint) =>
11541156
lookupTableOrView(identifier).map {
1155-
case v: ResolvedView => v
1156-
case _ =>
1157-
u.failAnalysis(s"${identifier.quoted} is a table. '$cmd' expects a view." +
1158-
relationTypeMismatchHint.map(" " + _).getOrElse(""))
1157+
case t: ResolvedTable =>
1158+
throw QueryCompilationErrors.expectViewNotTableError(
1159+
t, cmd, relationTypeMismatchHint, u)
1160+
case view => view
11591161
}.getOrElse(u)
11601162

11611163
case u @ UnresolvedTableOrView(identifier, _, _) =>

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,8 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod
4040
*/
4141
case class UnresolvedTable(
4242
multipartIdentifier: Seq[String],
43-
commandName: String) extends LeafNode {
43+
commandName: String,
44+
relationTypeMismatchHint: Option[String]) extends LeafNode {
4445
override lazy val resolved: Boolean = false
4546

4647
override def output: Seq[Attribute] = Nil

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

Lines changed: 21 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -2170,8 +2170,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
21702170
*/
21712171
private def createUnresolvedTable(
21722172
ctx: MultipartIdentifierContext,
2173-
commandName: String): UnresolvedTable = withOrigin(ctx) {
2174-
UnresolvedTable(visitMultipartIdentifier(ctx), commandName)
2173+
commandName: String,
2174+
relationTypeMismatchHint: Option[String] = None): UnresolvedTable = withOrigin(ctx) {
2175+
UnresolvedTable(visitMultipartIdentifier(ctx), commandName, relationTypeMismatchHint)
21752176
}
21762177

21772178
/**
@@ -3494,7 +3495,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
34943495
ctx.multipartIdentifier,
34953496
commandName = "ALTER VIEW ... SET TBLPROPERTIES",
34963497
allowTemp = false,
3497-
relationTypeMismatchHint = Some("Please use ALTER TABLE instead.")),
3498+
relationTypeMismatchHint = alterViewTypeMismatchHint),
34983499
cleanedTableProperties)
34993500
} else {
35003501
AlterTableSetPropertiesStatement(
@@ -3524,7 +3525,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
35243525
ctx.multipartIdentifier,
35253526
commandName = "ALTER VIEW ... UNSET TBLPROPERTIES",
35263527
allowTemp = false,
3527-
relationTypeMismatchHint = Some("Please use ALTER TABLE instead.")),
3528+
relationTypeMismatchHint = alterViewTypeMismatchHint),
35283529
cleanedProperties,
35293530
ifExists)
35303531
} else {
@@ -3547,7 +3548,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
35473548
AlterTableSetLocation(
35483549
createUnresolvedTable(
35493550
ctx.multipartIdentifier,
3550-
"ALTER TABLE ... SET LOCATION ..."),
3551+
"ALTER TABLE ... SET LOCATION ...",
3552+
alterTableTypeMismatchHint),
35513553
Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec),
35523554
visitLocationSpec(ctx.locationSpec))
35533555
}
@@ -3814,7 +3816,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
38143816
AlterTableRecoverPartitions(
38153817
createUnresolvedTable(
38163818
ctx.multipartIdentifier,
3817-
"ALTER TABLE ... RECOVER PARTITIONS"))
3819+
"ALTER TABLE ... RECOVER PARTITIONS",
3820+
alterTableTypeMismatchHint))
38183821
}
38193822

38203823
/**
@@ -3843,7 +3846,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
38433846
AlterTableAddPartition(
38443847
createUnresolvedTable(
38453848
ctx.multipartIdentifier,
3846-
"ALTER TABLE ... ADD PARTITION ..."),
3849+
"ALTER TABLE ... ADD PARTITION ...",
3850+
alterTableTypeMismatchHint),
38473851
specsAndLocs.toSeq,
38483852
ctx.EXISTS != null)
38493853
}
@@ -3861,7 +3865,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
38613865
AlterTableRenamePartition(
38623866
createUnresolvedTable(
38633867
ctx.multipartIdentifier,
3864-
"ALTER TABLE ... RENAME TO PARTITION"),
3868+
"ALTER TABLE ... RENAME TO PARTITION",
3869+
alterTableTypeMismatchHint),
38653870
UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.from)),
38663871
UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.to)))
38673872
}
@@ -3889,7 +3894,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
38893894
AlterTableDropPartition(
38903895
createUnresolvedTable(
38913896
ctx.multipartIdentifier,
3892-
"ALTER TABLE ... DROP PARTITION ..."),
3897+
"ALTER TABLE ... DROP PARTITION ...",
3898+
alterTableTypeMismatchHint),
38933899
partSpecs.toSeq,
38943900
ifExists = ctx.EXISTS != null,
38953901
purge = ctx.PURGE != null)
@@ -3909,7 +3915,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
39093915
AlterTableSerDeProperties(
39103916
createUnresolvedTable(
39113917
ctx.multipartIdentifier,
3912-
"ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
3918+
"ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]",
3919+
alterTableTypeMismatchHint),
39133920
Option(ctx.STRING).map(string),
39143921
Option(ctx.tablePropertyList).map(visitPropertyKeyValues),
39153922
// TODO a partition spec is allowed to have optional values. This is currently violated.
@@ -4122,4 +4129,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
41224129
}
41234130
CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
41244131
}
4132+
4133+
private def alterViewTypeMismatchHint: Option[String] = Some("Please use ALTER TABLE instead.")
4134+
4135+
private def alterTableTypeMismatchHint: Option[String] = Some("Please use ALTER VIEW instead.")
41254136
}

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path
2121

2222
import org.apache.spark.sql.AnalysisException
2323
import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier}
24-
import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedView}
24+
import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable, ResolvedView}
2525
import org.apache.spark.sql.catalyst.catalog.InvalidUDFClassException
2626
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateMap, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition}
2727
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SerdeInfo}
@@ -231,9 +231,18 @@ private[spark] object QueryCompilationErrors {
231231
t.origin.line, t.origin.startPosition)
232232
}
233233

234-
def expectTableNotViewError(v: ResolvedView, cmd: String, t: TreeNode[_]): Throwable = {
234+
def expectTableNotViewError(
235+
v: ResolvedView, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = {
235236
val viewStr = if (v.isTemp) "temp view" else "view"
236-
new AnalysisException(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.",
237+
val hintStr = mismatchHint.map(" " + _).getOrElse("")
238+
new AnalysisException(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.$hintStr",
239+
t.origin.line, t.origin.startPosition)
240+
}
241+
242+
def expectViewNotTableError(
243+
v: ResolvedTable, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = {
244+
val hintStr = mismatchHint.map(" " + _).getOrElse("")
245+
new AnalysisException(s"${v.identifier.quoted} is a table. '$cmd' expects a view.$hintStr",
237246
t.origin.line, t.origin.startPosition)
238247
}
239248

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

Lines changed: 34 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -867,17 +867,18 @@ class DDLParserSuite extends AnalysisTest {
867867
}
868868

869869
test("alter table: set location") {
870+
val hint = Some("Please use ALTER VIEW instead.")
870871
comparePlans(
871872
parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"),
872873
AlterTableSetLocation(
873-
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ..."),
874+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", hint),
874875
None,
875876
"new location"))
876877

877878
comparePlans(
878879
parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"),
879880
AlterTableSetLocation(
880-
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ..."),
881+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", hint),
881882
Some(Map("ds" -> "2017-06-10")),
882883
"new location"))
883884
}
@@ -1915,21 +1916,36 @@ class DDLParserSuite extends AnalysisTest {
19151916
test("MSCK REPAIR TABLE") {
19161917
comparePlans(
19171918
parsePlan("MSCK REPAIR TABLE a.b.c"),
1918-
RepairTable(UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE")))
1919+
RepairTable(UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE", None)))
19191920
}
19201921

19211922
test("LOAD DATA INTO table") {
19221923
comparePlans(
19231924
parsePlan("LOAD DATA INPATH 'filepath' INTO TABLE a.b.c"),
1924-
LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", false, false, None))
1925+
LoadData(
1926+
UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
1927+
"filepath",
1928+
false,
1929+
false,
1930+
None))
19251931

19261932
comparePlans(
19271933
parsePlan("LOAD DATA LOCAL INPATH 'filepath' INTO TABLE a.b.c"),
1928-
LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", true, false, None))
1934+
LoadData(
1935+
UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
1936+
"filepath",
1937+
true,
1938+
false,
1939+
None))
19291940

19301941
comparePlans(
19311942
parsePlan("LOAD DATA LOCAL INPATH 'filepath' OVERWRITE INTO TABLE a.b.c"),
1932-
LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", true, true, None))
1943+
LoadData(
1944+
UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
1945+
"filepath",
1946+
true,
1947+
true,
1948+
None))
19331949

19341950
comparePlans(
19351951
parsePlan(
@@ -1938,7 +1954,7 @@ class DDLParserSuite extends AnalysisTest {
19381954
|PARTITION(ds='2017-06-10')
19391955
""".stripMargin),
19401956
LoadData(
1941-
UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"),
1957+
UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
19421958
"filepath",
19431959
true,
19441960
true,
@@ -2003,12 +2019,12 @@ class DDLParserSuite extends AnalysisTest {
20032019
test("TRUNCATE table") {
20042020
comparePlans(
20052021
parsePlan("TRUNCATE TABLE a.b.c"),
2006-
TruncateTable(UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE"), None))
2022+
TruncateTable(UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE", None), None))
20072023

20082024
comparePlans(
20092025
parsePlan("TRUNCATE TABLE a.b.c PARTITION(ds='2017-06-10')"),
20102026
TruncateTable(
2011-
UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE"),
2027+
UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE", None),
20122028
Some(Map("ds" -> "2017-06-10"))))
20132029
}
20142030

@@ -2058,9 +2074,10 @@ class DDLParserSuite extends AnalysisTest {
20582074

20592075
test("alter table: SerDe properties") {
20602076
val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'"
2077+
val hint = Some("Please use ALTER VIEW instead.")
20612078
val parsed1 = parsePlan(sql1)
20622079
val expected1 = AlterTableSerDeProperties(
2063-
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2080+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
20642081
Some("org.apache.class"),
20652082
None,
20662083
None)
@@ -2073,7 +2090,7 @@ class DDLParserSuite extends AnalysisTest {
20732090
""".stripMargin
20742091
val parsed2 = parsePlan(sql2)
20752092
val expected2 = AlterTableSerDeProperties(
2076-
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2093+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
20772094
Some("org.apache.class"),
20782095
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
20792096
None)
@@ -2086,7 +2103,7 @@ class DDLParserSuite extends AnalysisTest {
20862103
""".stripMargin
20872104
val parsed3 = parsePlan(sql3)
20882105
val expected3 = AlterTableSerDeProperties(
2089-
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2106+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
20902107
None,
20912108
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
20922109
None)
@@ -2100,7 +2117,7 @@ class DDLParserSuite extends AnalysisTest {
21002117
""".stripMargin
21012118
val parsed4 = parsePlan(sql4)
21022119
val expected4 = AlterTableSerDeProperties(
2103-
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2120+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
21042121
Some("org.apache.class"),
21052122
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
21062123
Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
@@ -2113,7 +2130,7 @@ class DDLParserSuite extends AnalysisTest {
21132130
""".stripMargin
21142131
val parsed5 = parsePlan(sql5)
21152132
val expected5 = AlterTableSerDeProperties(
2116-
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2133+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
21172134
None,
21182135
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
21192136
Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
@@ -2126,7 +2143,7 @@ class DDLParserSuite extends AnalysisTest {
21262143
""".stripMargin
21272144
val parsed6 = parsePlan(sql6)
21282145
val expected6 = AlterTableSerDeProperties(
2129-
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2146+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
21302147
Some("org.apache.class"),
21312148
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
21322149
None)
@@ -2139,7 +2156,7 @@ class DDLParserSuite extends AnalysisTest {
21392156
""".stripMargin
21402157
val parsed7 = parsePlan(sql7)
21412158
val expected7 = AlterTableSerDeProperties(
2142-
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
2159+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint),
21432160
None,
21442161
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
21452162
Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
@@ -2470,7 +2487,7 @@ class DDLParserSuite extends AnalysisTest {
24702487

24712488
comparePlans(
24722489
parsePlan("COMMENT ON TABLE a.b.c IS 'xYz'"),
2473-
CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE"), "xYz"))
2490+
CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE", None), "xYz"))
24742491
}
24752492

24762493
test("create table - without using") {

sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -448,7 +448,8 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
448448
override def recoverPartitions(tableName: String): Unit = {
449449
val multiPartIdent = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(tableName)
450450
sparkSession.sessionState.executePlan(
451-
AlterTableRecoverPartitions(UnresolvedTable(multiPartIdent, "recoverPartitions()"))).toRdd
451+
AlterTableRecoverPartitions(
452+
UnresolvedTable(multiPartIdent, "recoverPartitions()", None))).toRdd
452453
}
453454

454455
/**

sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,10 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes
3030
|(dt='2009-09-09', country='uk')""".stripMargin
3131
val parsed = parsePlan(sql)
3232
val expected = AlterTableAddPartition(
33-
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ..."),
33+
UnresolvedTable(
34+
Seq("a", "b", "c"),
35+
"ALTER TABLE ... ADD PARTITION ...",
36+
Some("Please use ALTER VIEW instead.")),
3437
Seq(
3538
UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")),
3639
UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"), None)),
@@ -42,7 +45,10 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes
4245
val sql = "ALTER TABLE a.b.c ADD PARTITION (dt='2008-08-08') LOCATION 'loc'"
4346
val parsed = parsePlan(sql)
4447
val expected = AlterTableAddPartition(
45-
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ..."),
48+
UnresolvedTable(
49+
Seq("a", "b", "c"),
50+
"ALTER TABLE ... ADD PARTITION ...",
51+
Some("Please use ALTER VIEW instead.")),
4652
Seq(UnresolvedPartitionSpec(Map("dt" -> "2008-08-08"), Some("loc"))),
4753
ifNotExists = false)
4854

0 commit comments

Comments
 (0)