Skip to content

Commit 1c7b79c

Browse files
imback82cloud-fan
authored andcommitted
[SPARK-33856][SQL] Migrate ALTER TABLE ... RENAME TO PARTITION to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request? This PR proposes to migrate `ALTER TABLE ... RENAME TO PARTITION` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing). Note that `ALTER TABLE ... RENAME TO PARTITION` is not supported for v2 tables. ### Why are the changes needed? The PR makes the resolution consistent behavior consistent. For example, ``` sql("CREATE DATABASE test") sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)") sql("CREATE TEMPORARY VIEW t AS SELECT 2") sql("USE spark_catalog.test") sql("ALTER TABLE t PARTITION (id=1) RENAME TO PARTITION (id=2)") // works fine assuming id=1 exists. ``` , but after this PR: ``` sql("ALTER TABLE t PARTITION (id=1) RENAME TO PARTITION (id=2)") org.apache.spark.sql.AnalysisException: t is a temp view. 'ALTER TABLE ... RENAME TO PARTITION' expects a table; line 1 pos 0 ``` , which is the consistent behavior with other commands. ### Does this PR introduce _any_ user-facing change? After this PR, `ALTER TABLE` in the above example is resolved to a temp view `t` first instead of `spark_catalog.test.t`. ### How was this patch tested? Updated existing tests. Closes #30862 from imback82/alter_table_rename_partition_v2. Authored-by: Terry Kim <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 8e26339 commit 1c7b79c

File tree

12 files changed

+69
-34
lines changed

12 files changed

+69
-34
lines changed

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

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -587,6 +587,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
587587
case AlterTableDropPartition(ResolvedTable(_, _, table), parts, _, _) =>
588588
checkAlterTablePartition(table, parts)
589589

590+
case AlterTableRenamePartition(ResolvedTable(_, _, table), from, _) =>
591+
checkAlterTablePartition(table, Seq(from))
592+
590593
case showPartitions: ShowPartitions => checkShowPartitions(showPartitions)
591594

592595
case _ => // Falls back to the following checks

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

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis
2020
import org.apache.spark.sql.catalyst.InternalRow
2121
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
2222
import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
23-
import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, LogicalPlan, ShowPartitions}
23+
import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, AlterTableRenamePartition, LogicalPlan, ShowPartitions}
2424
import org.apache.spark.sql.catalyst.rules.Rule
2525
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
2626
import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement
@@ -51,6 +51,15 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
5151
partitionSchema,
5252
requireExactMatchedPartitionSpec(table.name, _, partitionSchema.fieldNames)))
5353

54+
case r @ AlterTableRenamePartition(
55+
ResolvedTable(_, _, table: SupportsPartitionManagement), from, _) =>
56+
val partitionSchema = table.partitionSchema()
57+
r.copy(from = resolvePartitionSpecs(
58+
table.name,
59+
Seq(from),
60+
partitionSchema,
61+
requireExactMatchedPartitionSpec(table.name, _, partitionSchema.fieldNames)).head)
62+
5463
case r @ ShowPartitions(ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs) =>
5564
r.copy(pattern = resolvePartitionSpecs(
5665
table.name,

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

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -3761,7 +3761,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
37613761
}
37623762

37633763
/**
3764-
* Create an [[AlterTableRenamePartitionStatement]]
3764+
* Create an [[AlterTableRenamePartition]]
37653765
*
37663766
* For example:
37673767
* {{{
@@ -3770,9 +3770,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
37703770
*/
37713771
override def visitRenameTablePartition(
37723772
ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) {
3773-
AlterTableRenamePartitionStatement(
3774-
visitMultipartIdentifier(ctx.multipartIdentifier),
3775-
visitNonOptionalPartitionSpec(ctx.from),
3773+
AlterTableRenamePartition(
3774+
UnresolvedTable(
3775+
visitMultipartIdentifier(ctx.multipartIdentifier),
3776+
"ALTER TABLE ... RENAME TO PARTITION"),
3777+
UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.from)),
37763778
visitNonOptionalPartitionSpec(ctx.to))
37773779
}
37783780

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

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -292,14 +292,6 @@ case class AlterTableSetLocationStatement(
292292
partitionSpec: Option[TablePartitionSpec],
293293
location: String) extends ParsedStatement
294294

295-
/**
296-
* ALTER TABLE ... RENAME PARTITION command, as parsed from SQL.
297-
*/
298-
case class AlterTableRenamePartitionStatement(
299-
tableName: Seq[String],
300-
from: TablePartitionSpec,
301-
to: TablePartitionSpec) extends ParsedStatement
302-
303295
/**
304296
* An INSERT INTO statement, as parsed from SQL.
305297
*

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

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -673,6 +673,19 @@ case class AlterTableDropPartition(
673673
override def children: Seq[LogicalPlan] = child :: Nil
674674
}
675675

676+
/**
677+
* The logical plan of the ALTER TABLE ... RENAME TO PARTITION command.
678+
*/
679+
case class AlterTableRenamePartition(
680+
child: LogicalPlan,
681+
from: PartitionSpec,
682+
to: TablePartitionSpec) extends Command {
683+
override lazy val resolved: Boolean =
684+
childrenResolved && from.isInstanceOf[ResolvedPartitionSpec]
685+
686+
override def children: Seq[LogicalPlan] = child :: Nil
687+
}
688+
676689
/**
677690
* The logical plan of the ALTER TABLE ... RECOVER PARTITIONS command.
678691
*/

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

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser
2020
import java.util.Locale
2121

2222
import org.apache.spark.sql.AnalysisException
23-
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedFunc, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView, UnresolvedView}
23+
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedFunc, UnresolvedNamespace, UnresolvedPartitionSpec, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView, UnresolvedView}
2424
import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, BucketSpec, FileResource, FunctionResource, JarResource}
2525
import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal}
2626
import org.apache.spark.sql.catalyst.plans.logical._
@@ -2106,9 +2106,9 @@ class DDLParserSuite extends AnalysisTest {
21062106
|RENAME TO PARTITION (dt='2008-09-09', country='uk')
21072107
""".stripMargin
21082108
val parsed1 = parsePlan(sql1)
2109-
val expected1 = AlterTableRenamePartitionStatement(
2110-
Seq("table_name"),
2111-
Map("dt" -> "2008-08-08", "country" -> "us"),
2109+
val expected1 = AlterTableRenamePartition(
2110+
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME TO PARTITION"),
2111+
UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
21122112
Map("dt" -> "2008-09-09", "country" -> "uk"))
21132113
comparePlans(parsed1, expected1)
21142114

@@ -2118,9 +2118,9 @@ class DDLParserSuite extends AnalysisTest {
21182118
|RENAME TO PARTITION (ds='2018-06-10')
21192119
""".stripMargin
21202120
val parsed2 = parsePlan(sql2)
2121-
val expected2 = AlterTableRenamePartitionStatement(
2122-
Seq("a", "b", "c"),
2123-
Map("ds" -> "2017-06-10"),
2121+
val expected2 = AlterTableRenamePartition(
2122+
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO PARTITION"),
2123+
UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")),
21242124
Map("ds" -> "2018-06-10"))
21252125
comparePlans(parsed2, expected2)
21262126
}

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -445,10 +445,10 @@ class ResolveSessionCatalog(
445445
partSpecsAndLocs.asUnresolvedPartitionSpecs.map(spec => (spec.spec, spec.location)),
446446
ifNotExists)
447447

448-
case AlterTableRenamePartitionStatement(tbl, from, to) =>
449-
val v1TableName = parseV1Table(tbl, "ALTER TABLE RENAME PARTITION")
448+
case AlterTableRenamePartition(
449+
ResolvedV1TableIdentifier(ident), UnresolvedPartitionSpec(from, _), to) =>
450450
AlterTableRenamePartitionCommand(
451-
v1TableName.asTableIdentifier,
451+
ident.asTableIdentifier,
452452
from,
453453
to)
454454

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -342,6 +342,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
342342
AlterTableDropPartitionExec(
343343
table, parts.asResolvedPartitionSpecs, ignoreIfNotExists) :: Nil
344344

345+
case AlterTableRenamePartition(_: ResolvedTable, _: ResolvedPartitionSpec, _) =>
346+
throw new AnalysisException(
347+
"ALTER TABLE ... RENAME TO PARTITION is not supported for v2 tables.")
348+
345349
case AlterTableRecoverPartitions(_: ResolvedTable) =>
346350
throw new AnalysisException(
347351
"ALTER TABLE ... RECOVER PARTITIONS is not supported for v2 tables.")

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

Lines changed: 14 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -33,13 +33,21 @@ class AlterTablePartitionV2SQLSuite extends DatasourceV2SQLBase {
3333
}
3434

3535
test("ALTER TABLE RENAME PARTITION") {
36-
val t = "testcat.ns1.ns2.tbl"
37-
withTable(t) {
38-
spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)")
39-
val e = intercept[AnalysisException] {
40-
sql(s"ALTER TABLE $t PARTITION (id=1) RENAME TO PARTITION (id=2)")
36+
val nonPartTbl = "testcat.ns1.ns2.tbl"
37+
val partTbl = "testpart.ns1.ns2.tbl"
38+
withTable(nonPartTbl, partTbl) {
39+
spark.sql(s"CREATE TABLE $nonPartTbl (id bigint, data string) USING foo PARTITIONED BY (id)")
40+
val e1 = intercept[AnalysisException] {
41+
sql(s"ALTER TABLE $nonPartTbl PARTITION (id=1) RENAME TO PARTITION (id=2)")
42+
}
43+
assert(e1.message.contains(s"Table $nonPartTbl can not alter partitions"))
44+
45+
spark.sql(s"CREATE TABLE $partTbl (id bigint, data string) USING foo PARTITIONED BY (id)")
46+
val e2 = intercept[AnalysisException] {
47+
sql(s"ALTER TABLE $partTbl PARTITION (id=1) RENAME TO PARTITION (id=2)")
4148
}
42-
assert(e.message.contains("ALTER TABLE RENAME PARTITION is only supported with v1 tables"))
49+
assert(e2.message.contains(
50+
"ALTER TABLE ... RENAME TO PARTITION is not supported for v2 tables."))
4351
}
4452
}
4553
}

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -149,7 +149,9 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
149149
assertAnalysisError(
150150
s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')",
151151
s"$viewName is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table")
152-
assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')")
152+
assertAnalysisError(
153+
s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')",
154+
s"$viewName is a temp view. 'ALTER TABLE ... RENAME TO PARTITION' expects a table")
153155
assertAnalysisError(
154156
s"ALTER TABLE $viewName RECOVER PARTITIONS",
155157
s"$viewName is a temp view. 'ALTER TABLE ... RECOVER PARTITIONS' expects a table")

0 commit comments

Comments
 (0)