Skip to content

Commit ae7450d

Browse files
huaxingaoviirya
authored andcommitted
[SPARK-29676][SQL] ALTER TABLE (RENAME PARTITION) should look up catalog/table like v2 commands
### What changes were proposed in this pull request? Add AlterTableRenamePartitionStatement and make ALTER TABLE ... RENAME TO PARTITION go through the same catalog/table resolution framework of v2 commands. ### 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 (id=1) RENAME TO PARTITION (id=2) // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running ALTER TABLE ... RENAME TO PARTITION, 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? Unit tests. Closes apache#26350 from huaxingao/spark_29676. Authored-by: Huaxin Gao <[email protected]> Signed-off-by: Liang-Chi Hsieh <[email protected]>
1 parent 3175f4b commit ae7450d

File tree

8 files changed

+73
-35
lines changed

8 files changed

+73
-35
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -162,7 +162,7 @@ statement
162162
partitionSpecLocation+ #addTablePartition
163163
| ALTER VIEW tableIdentifier ADD (IF NOT EXISTS)?
164164
partitionSpec+ #addTablePartition
165-
| ALTER TABLE tableIdentifier
165+
| ALTER TABLE multipartIdentifier
166166
from=partitionSpec RENAME TO to=partitionSpec #renameTablePartition
167167
| ALTER TABLE tableIdentifier
168168
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions

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

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2951,4 +2951,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
29512951
ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) {
29522952
AlterTableRecoverPartitionsStatement(visitMultipartIdentifier(ctx.multipartIdentifier))
29532953
}
2954+
2955+
/**
2956+
* Create an [[AlterTableRenamePartitionStatement]]
2957+
*
2958+
* For example:
2959+
* {{{
2960+
* ALTER TABLE multi_part_name PARTITION spec1 RENAME TO PARTITION spec2;
2961+
* }}}
2962+
*/
2963+
override def visitRenameTablePartition(
2964+
ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) {
2965+
AlterTableRenamePartitionStatement(
2966+
visitMultipartIdentifier(ctx.multipartIdentifier),
2967+
visitNonOptionalPartitionSpec(ctx.from),
2968+
visitNonOptionalPartitionSpec(ctx.to))
2969+
}
29542970
}

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

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -188,6 +188,14 @@ case class AlterTableSetLocationStatement(
188188
case class AlterTableRecoverPartitionsStatement(
189189
tableName: Seq[String]) extends ParsedStatement
190190

191+
/**
192+
* ALTER TABLE ... RENAME PARTITION command, as parsed from SQL.
193+
*/
194+
case class AlterTableRenamePartitionStatement(
195+
tableName: Seq[String],
196+
from: TablePartitionSpec,
197+
to: TablePartitionSpec) extends ParsedStatement
198+
191199
/**
192200
* ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL.
193201
*/

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

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1202,6 +1202,32 @@ class DDLParserSuite extends AnalysisTest {
12021202
AlterTableRecoverPartitionsStatement(Seq("a", "b", "c")))
12031203
}
12041204

1205+
test("alter table: rename partition") {
1206+
val sql1 =
1207+
"""
1208+
|ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us')
1209+
|RENAME TO PARTITION (dt='2008-09-09', country='uk')
1210+
""".stripMargin
1211+
val parsed1 = parsePlan(sql1)
1212+
val expected1 = AlterTableRenamePartitionStatement(
1213+
Seq("table_name"),
1214+
Map("dt" -> "2008-08-08", "country" -> "us"),
1215+
Map("dt" -> "2008-09-09", "country" -> "uk"))
1216+
comparePlans(parsed1, expected1)
1217+
1218+
val sql2 =
1219+
"""
1220+
|ALTER TABLE a.b.c PARTITION (ds='2017-06-10')
1221+
|RENAME TO PARTITION (ds='2018-06-10')
1222+
""".stripMargin
1223+
val parsed2 = parsePlan(sql2)
1224+
val expected2 = AlterTableRenamePartitionStatement(
1225+
Seq("a", "b", "c"),
1226+
Map("ds" -> "2017-06-10"),
1227+
Map("ds" -> "2018-06-10"))
1228+
comparePlans(parsed2, expected2)
1229+
}
1230+
12051231
private case class TableSpec(
12061232
name: Seq[String],
12071233
schema: Option[StructType],

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

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
2424
import org.apache.spark.sql.catalyst.rules.Rule
2525
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table}
2626
import org.apache.spark.sql.connector.expressions.Transform
27-
import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropDatabaseCommand, DropTableCommand, LoadDataCommand, ShowColumnsCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand}
27+
import org.apache.spark.sql.execution.command._
2828
import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable}
2929
import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
3030
import org.apache.spark.sql.internal.SQLConf
@@ -369,6 +369,13 @@ class ResolveSessionCatalog(
369369
AlterTableRecoverPartitionsCommand(
370370
v1TableName.asTableIdentifier,
371371
"ALTER TABLE RECOVER PARTITIONS")
372+
373+
case AlterTableRenamePartitionStatement(tableName, from, to) =>
374+
val v1TableName = parseV1Table(tableName, "ALTER TABLE RENAME PARTITION")
375+
AlterTableRenamePartitionCommand(
376+
v1TableName.asTableIdentifier,
377+
from,
378+
to)
372379
}
373380

374381
private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = {

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
@@ -461,22 +461,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
461461
ctx.EXISTS != null)
462462
}
463463

464-
/**
465-
* Create an [[AlterTableRenamePartitionCommand]] command
466-
*
467-
* For example:
468-
* {{{
469-
* ALTER TABLE table PARTITION spec1 RENAME TO PARTITION spec2;
470-
* }}}
471-
*/
472-
override def visitRenameTablePartition(
473-
ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) {
474-
AlterTableRenamePartitionCommand(
475-
visitTableIdentifier(ctx.tableIdentifier),
476-
visitNonOptionalPartitionSpec(ctx.from),
477-
visitNonOptionalPartitionSpec(ctx.to))
478-
}
479-
480464
/**
481465
* Create an [[AlterTableDropPartitionCommand]] command
482466
*

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

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1374,16 +1374,27 @@ class DataSourceV2SQLSuite
13741374
}
13751375
}
13761376

1377-
test("ALTER TABLE RECOVER PARTITIONS") {
1377+
test("ALTER TABLE RECOVER PARTITIONS") {
13781378
val t = "testcat.ns1.ns2.tbl"
13791379
withTable(t) {
13801380
spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo")
13811381
val e = intercept[AnalysisException] {
1382-
val partition = sql(s"ALTER TABLE $t RECOVER PARTITIONS")
1382+
sql(s"ALTER TABLE $t RECOVER PARTITIONS")
13831383
}
13841384
assert(e.message.contains("ALTER TABLE RECOVER PARTITIONS is only supported with v1 tables"))
13851385
}
1386-
}
1386+
}
1387+
1388+
test("ALTER TABLE RENAME PARTITION") {
1389+
val t = "testcat.ns1.ns2.tbl"
1390+
withTable(t) {
1391+
spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)")
1392+
val e = intercept[AnalysisException] {
1393+
sql(s"ALTER TABLE $t PARTITION (id=1) RENAME TO PARTITION (id=2)")
1394+
}
1395+
assert(e.message.contains("ALTER TABLE RENAME PARTITION is only supported with v1 tables"))
1396+
}
1397+
}
13871398

13881399
private def testV1Command(sqlCommand: String, sqlParams: String): Unit = {
13891400
val e = intercept[AnalysisException] {

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

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -558,20 +558,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession {
558558
""".stripMargin)
559559
}
560560

561-
test("alter table: rename partition") {
562-
val sql =
563-
"""
564-
|ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us')
565-
|RENAME TO PARTITION (dt='2008-09-09', country='uk')
566-
""".stripMargin
567-
val parsed = parser.parsePlan(sql)
568-
val expected = AlterTableRenamePartitionCommand(
569-
TableIdentifier("table_name", None),
570-
Map("dt" -> "2008-08-08", "country" -> "us"),
571-
Map("dt" -> "2008-09-09", "country" -> "uk"))
572-
comparePlans(parsed, expected)
573-
}
574-
575561
test("alter table: exchange partition (not supported)") {
576562
assertUnsupported(
577563
"""

0 commit comments

Comments
 (0)