Skip to content

Commit 3a06c12

Browse files
imback82cloud-fan
authored andcommitted
[SPARK-29592][SQL] ALTER TABLE (set partition location) should look up catalog/table like v2 commands
### What changes were proposed in this pull request? Update `AlterTableSetLocationStatement` to store `partitionSpec` and make `ALTER TABLE a.b.c PARTITION(...) SET LOCATION 'loc'` fail if `partitionSpec` is set with unsupported message. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog ALTER TABLE t PARTITION(...) SET LOCATION 'loc' // report set location with partition spec is not supported. ``` ### Does this PR introduce any user-facing change? yes. When running ALTER TABLE (set partition location), Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? New unit tests Closes #26304 from imback82/alter_table_partition_loc. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent 401a5f7 commit 3a06c12

File tree

10 files changed

+49
-41
lines changed

10 files changed

+49
-41
lines changed

sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -168,8 +168,8 @@ statement
168168
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions
169169
| ALTER VIEW tableIdentifier
170170
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions
171-
| ALTER TABLE multipartIdentifier SET locationSpec #setTableLocation
172-
| ALTER TABLE tableIdentifier partitionSpec SET locationSpec #setPartitionLocation
171+
| ALTER TABLE multipartIdentifier
172+
(partitionSpec)? SET locationSpec #setTableLocation
173173
| ALTER TABLE multipartIdentifier RECOVER PARTITIONS #recoverPartitions
174174
| DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable
175175
| DROP VIEW (IF EXISTS)? multipartIdentifier #dropView

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

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -73,7 +73,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
7373
createAlterTable(nameParts, catalog, tableName, changes)
7474

7575
case AlterTableSetLocationStatement(
76-
nameParts @ NonSessionCatalog(catalog, tableName), newLoc) =>
76+
nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, newLoc) =>
77+
if (partitionSpec.nonEmpty) {
78+
throw new AnalysisException(
79+
"ALTER TABLE SET LOCATION does not support partition for v2 tables.")
80+
}
7781
val changes = Seq(TableChange.setProperty("location", newLoc))
7882
createAlterTable(nameParts, catalog, tableName, changes)
7983

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2727,6 +2727,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
27272727
override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) {
27282728
AlterTableSetLocationStatement(
27292729
visitMultipartIdentifier(ctx.multipartIdentifier),
2730+
Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec),
27302731
visitLocationSpec(ctx.locationSpec))
27312732
}
27322733

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -179,6 +179,7 @@ case class AlterTableUnsetPropertiesStatement(
179179
*/
180180
case class AlterTableSetLocationStatement(
181181
tableName: Seq[String],
182+
partitionSpec: Option[TablePartitionSpec],
182183
location: String) extends ParsedStatement
183184

184185
/**

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

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -540,10 +540,16 @@ class DDLParserSuite extends AnalysisTest {
540540
}
541541

542542
test("alter table: set location") {
543-
val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'"
544-
val parsed1 = parsePlan(sql1)
545-
val expected1 = AlterTableSetLocationStatement(Seq("table_name"), "new location")
546-
comparePlans(parsed1, expected1)
543+
comparePlans(
544+
parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"),
545+
AlterTableSetLocationStatement(Seq("a", "b", "c"), None, "new location"))
546+
547+
comparePlans(
548+
parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"),
549+
AlterTableSetLocationStatement(
550+
Seq("a", "b", "c"),
551+
Some(Map("ds" -> "2017-06-10")),
552+
"new location"))
547553
}
548554

549555
test("alter table: rename column") {

sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -118,11 +118,15 @@ class ResolveSessionCatalog(
118118
}
119119

120120
case AlterTableSetLocationStatement(
121-
nameParts @ SessionCatalog(catalog, tableName), newLoc) =>
121+
nameParts @ SessionCatalog(catalog, tableName), partitionSpec, newLoc) =>
122122
loadTable(catalog, tableName.asIdentifier).collect {
123123
case v1Table: V1Table =>
124-
AlterTableSetLocationCommand(tableName.asTableIdentifier, None, newLoc)
124+
AlterTableSetLocationCommand(tableName.asTableIdentifier, partitionSpec, newLoc)
125125
}.getOrElse {
126+
if (partitionSpec.nonEmpty) {
127+
throw new AnalysisException(
128+
"ALTER TABLE SET LOCATION does not support partition for v2 tables.")
129+
}
126130
val changes = Seq(TableChange.setProperty("location", newLoc))
127131
createAlterTable(nameParts, catalog, tableName, changes)
128132
}

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala

Lines changed: 0 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -502,22 +502,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
502502
retainData = false)
503503
}
504504

505-
/**
506-
* Create an [[AlterTableSetLocationCommand]] command for a partition.
507-
*
508-
* For example:
509-
* {{{
510-
* ALTER TABLE table PARTITION spec SET LOCATION "loc";
511-
* }}}
512-
*/
513-
override def visitSetPartitionLocation(
514-
ctx: SetPartitionLocationContext): LogicalPlan = withOrigin(ctx) {
515-
AlterTableSetLocationCommand(
516-
visitTableIdentifier(ctx.tableIdentifier),
517-
Some(visitNonOptionalPartitionSpec(ctx.partitionSpec)),
518-
visitLocationSpec(ctx.locationSpec))
519-
}
520-
521505
/**
522506
* Create a [[AlterTableChangeColumnCommand]] command.
523507
*

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

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -816,6 +816,19 @@ trait AlterTableTests extends SharedSparkSession {
816816
}
817817
}
818818

819+
test("AlterTable: set partition location") {
820+
val t = s"${catalogAndNamespace}table_name"
821+
withTable(t) {
822+
sql(s"CREATE TABLE $t (id int) USING $v2Format")
823+
824+
val exc = intercept[AnalysisException] {
825+
sql(s"ALTER TABLE $t PARTITION(ds='2017-06-10') SET LOCATION 's3://bucket/path'")
826+
}
827+
assert(exc.getMessage.contains(
828+
"ALTER TABLE SET LOCATION does not support partition for v2 tables"))
829+
}
830+
}
831+
819832
test("AlterTable: set table property") {
820833
val t = s"${catalogAndNamespace}table_name"
821834
withTable(t) {

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

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -136,14 +136,21 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
136136
assertNoSuchTable(s"ALTER TABLE $viewName SET SERDE 'whatever'")
137137
assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'")
138138
assertNoSuchTable(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')")
139-
assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'")
140139
assertNoSuchTable(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')")
141140
assertNoSuchTable(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')")
142141
assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')")
143142
assertNoSuchTable(s"ALTER TABLE $viewName RECOVER PARTITIONS")
144143

145144
// For v2 ALTER TABLE statements, we have better error message saying view is not supported.
146-
assertViewNotSupported(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'")
145+
assertAnalysisError(
146+
s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'",
147+
s"'$viewName' is a view not a table")
148+
149+
// For the following v2 ALERT TABLE statements, unsupported operations are checked first
150+
// before resolving the relations.
151+
assertAnalysisError(
152+
s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'",
153+
"ALTER TABLE SET LOCATION does not support partition for v2 tables")
147154
}
148155
}
149156

@@ -177,9 +184,9 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
177184
}
178185
}
179186

180-
private def assertViewNotSupported(query: String): Unit = {
187+
private def assertAnalysisError(query: String, message: String): Unit = {
181188
val e = intercept[AnalysisException](sql(query))
182-
assert(e.message.contains("'testView' is a view not a table"))
189+
assert(e.message.contains(message))
183190
}
184191

185192
test("error handling: insert/load/truncate table commands against a view") {

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

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -635,18 +635,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession {
635635
"SET FILEFORMAT PARQUET")
636636
}
637637

638-
test("alter table: set partition location") {
639-
val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " +
640-
"SET LOCATION 'new location'"
641-
val parsed2 = parser.parsePlan(sql2)
642-
val tableIdent = TableIdentifier("table_name", None)
643-
val expected2 = AlterTableSetLocationCommand(
644-
tableIdent,
645-
Some(Map("dt" -> "2008-08-08", "country" -> "us")),
646-
"new location")
647-
comparePlans(parsed2, expected2)
648-
}
649-
650638
test("alter table: change column name/type/comment") {
651639
val sql1 = "ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT"
652640
val sql2 = "ALTER TABLE table_name CHANGE COLUMN col_name col_name INT COMMENT 'new_comment'"

0 commit comments

Comments
 (0)