From e2488bb5b598ee478557a63ca1e96d2c223a6ce7 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 19 Dec 2020 22:57:44 -0800 Subject: [PATCH 1/2] initial commit --- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 8 +++++--- .../spark/sql/catalyst/plans/logical/statements.scala | 8 -------- .../spark/sql/catalyst/plans/logical/v2Commands.scala | 10 ++++++++++ .../spark/sql/catalyst/parser/DDLParserSuite.scala | 8 ++++---- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 5 ++--- .../datasources/v2/DataSourceV2Strategy.scala | 4 ++++ .../sql/connector/AlterTablePartitionV2SQLSuite.scala | 3 ++- .../org/apache/spark/sql/execution/SQLViewSuite.scala | 4 +++- .../apache/spark/sql/execution/command/DDLSuite.scala | 5 +++-- .../apache/spark/sql/hive/execution/HiveDDLSuite.scala | 5 +++-- 10 files changed, 36 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 94589688953d..2c5b052ed36c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3761,7 +3761,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableRenamePartitionStatement]] + * Create an [[AlterTableRenamePartition]] * * For example: * {{{ @@ -3770,8 +3770,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitRenameTablePartition( ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenamePartitionStatement( - visitMultipartIdentifier(ctx.multipartIdentifier), + AlterTableRenamePartition( + UnresolvedTable( + visitMultipartIdentifier(ctx.multipartIdentifier), + "ALTER TABLE ... RENAME TO PARTITION"), visitNonOptionalPartitionSpec(ctx.from), visitNonOptionalPartitionSpec(ctx.to)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 59239f6e041a..f6d141ded384 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -292,14 +292,6 @@ case class AlterTableSetLocationStatement( partitionSpec: Option[TablePartitionSpec], location: String) extends ParsedStatement -/** - * ALTER TABLE ... RENAME PARTITION command, as parsed from SQL. - */ -case class AlterTableRenamePartitionStatement( - tableName: Seq[String], - from: TablePartitionSpec, - to: TablePartitionSpec) extends ParsedStatement - /** * An INSERT INTO statement, as parsed from SQL. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index fa67d311c39c..e9d31cf8c459 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -673,6 +673,16 @@ case class AlterTableDropPartition( override def children: Seq[LogicalPlan] = child :: Nil } +/** + * The logical plan of the ALTER TABLE ... RENAME TO PARTITION command. + */ +case class AlterTableRenamePartition( + child: LogicalPlan, + from: TablePartitionSpec, + to: TablePartitionSpec) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} + /** * The logical plan of the ALTER TABLE ... RECOVER PARTITIONS command. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 5eb0c9a39f1e..ac67b4acbf90 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2106,8 +2106,8 @@ class DDLParserSuite extends AnalysisTest { |RENAME TO PARTITION (dt='2008-09-09', country='uk') """.stripMargin val parsed1 = parsePlan(sql1) - val expected1 = AlterTableRenamePartitionStatement( - Seq("table_name"), + val expected1 = AlterTableRenamePartition( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME TO PARTITION"), Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2008-09-09", "country" -> "uk")) comparePlans(parsed1, expected1) @@ -2118,8 +2118,8 @@ class DDLParserSuite extends AnalysisTest { |RENAME TO PARTITION (ds='2018-06-10') """.stripMargin val parsed2 = parsePlan(sql2) - val expected2 = AlterTableRenamePartitionStatement( - Seq("a", "b", "c"), + val expected2 = AlterTableRenamePartition( + UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO PARTITION"), Map("ds" -> "2017-06-10"), Map("ds" -> "2018-06-10")) comparePlans(parsed2, expected2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 723647a4a920..e1550f1d9d9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -445,10 +445,9 @@ class ResolveSessionCatalog( partSpecsAndLocs.asUnresolvedPartitionSpecs.map(spec => (spec.spec, spec.location)), ifNotExists) - case AlterTableRenamePartitionStatement(tbl, from, to) => - val v1TableName = parseV1Table(tbl, "ALTER TABLE RENAME PARTITION") + case AlterTableRenamePartition(ResolvedV1TableIdentifier(ident), from, to) => AlterTableRenamePartitionCommand( - v1TableName.asTableIdentifier, + ident.asTableIdentifier, from, to) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 50bcf81f1ba2..5e8b7fcd5922 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -342,6 +342,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat AlterTableDropPartitionExec( table, parts.asResolvedPartitionSpecs, ignoreIfNotExists) :: Nil + case AlterTableRenamePartition(_: ResolvedTable, _, _) => + throw new AnalysisException( + "ALTER TABLE ... RENAME TO PARTITION is not supported for v2 tables.") + case AlterTableRecoverPartitions(_: ResolvedTable) => throw new AnalysisException( "ALTER TABLE ... RECOVER PARTITIONS is not supported for v2 tables.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala index ac4d055eb0e6..d9e845544c71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala @@ -39,7 +39,8 @@ class AlterTablePartitionV2SQLSuite extends DatasourceV2SQLBase { val e = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION (id=1) RENAME TO PARTITION (id=2)") } - assert(e.message.contains("ALTER TABLE RENAME PARTITION is only supported with v1 tables")) + assert(e.message.contains( + "ALTER TABLE ... RENAME TO PARTITION is not supported for v2 tables.")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 6d65fddb1be6..9b84e0fe4bcb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -149,7 +149,9 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { assertAnalysisError( s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')", s"$viewName is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table") - assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") + assertAnalysisError( + s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')", + s"$viewName is a temp view. 'ALTER TABLE ... RENAME TO PARTITION' expects a table") assertAnalysisError( s"ALTER TABLE $viewName RECOVER PARTITIONS", s"$viewName is a temp view. 'ALTER TABLE ... RECOVER PARTITIONS' expects a table") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index d6474ae7d5f0..7a6076d6d957 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.internal.config import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchFunctionException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchFunctionException, NoSuchPartitionException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER @@ -1642,9 +1642,10 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Set(Map("a" -> "10", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) // table to alter does not exist - intercept[NoSuchTableException] { + val e = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist PARTITION (c='3') RENAME TO PARTITION (c='333')") } + assert(e.getMessage.contains("Table not found: does_not_exist")) // partition to rename does not exist intercept[NoSuchPartitionException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 34f127bade95..e55b2d390a5d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -896,8 +896,9 @@ class HiveDDLSuite s"ALTER TABLE $oldViewName RECOVER PARTITIONS", s"$oldViewName is a view. 'ALTER TABLE ... RECOVER PARTITIONS' expects a table.") - assertErrorForAlterTableOnView( - s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')") + assertAnalysisError( + s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')", + s"$oldViewName is a view. 'ALTER TABLE ... RENAME TO PARTITION' expects a table.") assertAnalysisError( s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')", From 9894f9eaf5806f4bfe5961ba3608a2344bef6942 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sun, 20 Dec 2020 18:12:04 -0800 Subject: [PATCH 2/2] Use UnresolvedPartitionSpec --- .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +++ .../analysis/ResolvePartitionSpec.scala | 11 ++++++++++- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../catalyst/plans/logical/v2Commands.scala | 5 ++++- .../sql/catalyst/parser/DDLParserSuite.scala | 6 +++--- .../analysis/ResolveSessionCatalog.scala | 3 ++- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../AlterTablePartitionV2SQLSuite.scala | 19 +++++++++++++------ 8 files changed, 37 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c0cdcdf2d957..472de096b2f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -587,6 +587,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case AlterTableDropPartition(ResolvedTable(_, _, table), parts, _, _) => checkAlterTablePartition(table, parts) + case AlterTableRenamePartition(ResolvedTable(_, _, table), from, _) => + checkAlterTablePartition(table, Seq(from)) + case showPartitions: ShowPartitions => checkShowPartitions(showPartitions) case _ => // Falls back to the following checks diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 35e4820cd710..2c2bea6f89d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, LogicalPlan, ShowPartitions} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, AlterTableRenamePartition, LogicalPlan, ShowPartitions} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement @@ -51,6 +51,15 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { partitionSchema, requireExactMatchedPartitionSpec(table.name, _, partitionSchema.fieldNames))) + case r @ AlterTableRenamePartition( + ResolvedTable(_, _, table: SupportsPartitionManagement), from, _) => + val partitionSchema = table.partitionSchema() + r.copy(from = resolvePartitionSpecs( + table.name, + Seq(from), + partitionSchema, + requireExactMatchedPartitionSpec(table.name, _, partitionSchema.fieldNames)).head) + case r @ ShowPartitions(ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs) => r.copy(pattern = resolvePartitionSpecs( table.name, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 2c5b052ed36c..9c265544f322 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3774,7 +3774,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg UnresolvedTable( visitMultipartIdentifier(ctx.multipartIdentifier), "ALTER TABLE ... RENAME TO PARTITION"), - visitNonOptionalPartitionSpec(ctx.from), + UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.from)), visitNonOptionalPartitionSpec(ctx.to)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index e9d31cf8c459..87d81d533057 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -678,8 +678,11 @@ case class AlterTableDropPartition( */ case class AlterTableRenamePartition( child: LogicalPlan, - from: TablePartitionSpec, + from: PartitionSpec, to: TablePartitionSpec) extends Command { + override lazy val resolved: Boolean = + childrenResolved && from.isInstanceOf[ResolvedPartitionSpec] + override def children: Seq[LogicalPlan] = child :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index ac67b4acbf90..330a01be4bfb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedFunc, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView, UnresolvedView} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedFunc, UnresolvedNamespace, UnresolvedPartitionSpec, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView, UnresolvedView} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, BucketSpec, FileResource, FunctionResource, JarResource} import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ @@ -2108,7 +2108,7 @@ class DDLParserSuite extends AnalysisTest { val parsed1 = parsePlan(sql1) val expected1 = AlterTableRenamePartition( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME TO PARTITION"), - Map("dt" -> "2008-08-08", "country" -> "us"), + UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")), Map("dt" -> "2008-09-09", "country" -> "uk")) comparePlans(parsed1, expected1) @@ -2120,7 +2120,7 @@ class DDLParserSuite extends AnalysisTest { val parsed2 = parsePlan(sql2) val expected2 = AlterTableRenamePartition( UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO PARTITION"), - Map("ds" -> "2017-06-10"), + UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")), Map("ds" -> "2018-06-10")) comparePlans(parsed2, expected2) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index e1550f1d9d9a..66d1c406a560 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -445,7 +445,8 @@ class ResolveSessionCatalog( partSpecsAndLocs.asUnresolvedPartitionSpecs.map(spec => (spec.spec, spec.location)), ifNotExists) - case AlterTableRenamePartition(ResolvedV1TableIdentifier(ident), from, to) => + case AlterTableRenamePartition( + ResolvedV1TableIdentifier(ident), UnresolvedPartitionSpec(from, _), to) => AlterTableRenamePartitionCommand( ident.asTableIdentifier, from, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 5e8b7fcd5922..635117a9932a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -342,7 +342,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat AlterTableDropPartitionExec( table, parts.asResolvedPartitionSpecs, ignoreIfNotExists) :: Nil - case AlterTableRenamePartition(_: ResolvedTable, _, _) => + case AlterTableRenamePartition(_: ResolvedTable, _: ResolvedPartitionSpec, _) => throw new AnalysisException( "ALTER TABLE ... RENAME TO PARTITION is not supported for v2 tables.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala index d9e845544c71..bdf2fa5b7ac9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTablePartitionV2SQLSuite.scala @@ -33,13 +33,20 @@ class AlterTablePartitionV2SQLSuite extends DatasourceV2SQLBase { } test("ALTER TABLE RENAME PARTITION") { - val t = "testcat.ns1.ns2.tbl" - withTable(t) { - spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)") - val e = intercept[AnalysisException] { - sql(s"ALTER TABLE $t PARTITION (id=1) RENAME TO PARTITION (id=2)") + val nonPartTbl = "testcat.ns1.ns2.tbl" + val partTbl = "testpart.ns1.ns2.tbl" + withTable(nonPartTbl, partTbl) { + spark.sql(s"CREATE TABLE $nonPartTbl (id bigint, data string) USING foo PARTITIONED BY (id)") + val e1 = intercept[AnalysisException] { + sql(s"ALTER TABLE $nonPartTbl PARTITION (id=1) RENAME TO PARTITION (id=2)") } - assert(e.message.contains( + assert(e1.message.contains(s"Table $nonPartTbl can not alter partitions")) + + spark.sql(s"CREATE TABLE $partTbl (id bigint, data string) USING foo PARTITIONED BY (id)") + val e2 = intercept[AnalysisException] { + sql(s"ALTER TABLE $partTbl PARTITION (id=1) RENAME TO PARTITION (id=2)") + } + assert(e2.message.contains( "ALTER TABLE ... RENAME TO PARTITION is not supported for v2 tables.")) } }