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 29bd2c256df8b..1d44e6ba298e1 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 @@ -565,13 +565,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // no validation needed for set and remove property } - case AlterTableAddPartition(r: ResolvedTable, parts, _) => + case AddPartitions(r: ResolvedTable, parts, _) => checkAlterTablePartition(r.table, parts) - case AlterTableDropPartition(r: ResolvedTable, parts, _, _) => + case DropPartitions(r: ResolvedTable, parts, _, _) => checkAlterTablePartition(r.table, parts) - case AlterTableRenamePartition(r: ResolvedTable, from, _) => + case RenamePartitions(r: ResolvedTable, from, _) => checkAlterTablePartition(r.table, Seq(from)) case showPartitions: ShowPartitions => checkShowPartitions(showPartitions) 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 2307152b17375..72298b285f2b6 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, AlterTableRenamePartition, LogicalPlan, ShowPartitions} +import org.apache.spark.sql.catalyst.plans.logical.{AddPartitions, DropPartitions, LogicalPlan, RenamePartitions, 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.util.PartitioningUtils.{normalizePartitionSpec, requ object ResolvePartitionSpec extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case r @ AlterTableAddPartition( + case r @ AddPartitions( ResolvedTable(_, _, table: SupportsPartitionManagement, _), partSpecs, _) => val partitionSchema = table.partitionSchema() r.copy(parts = resolvePartitionSpecs( @@ -42,7 +42,7 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { partitionSchema, requireExactMatchedPartitionSpec(table.name, _, partitionSchema.fieldNames))) - case r @ AlterTableDropPartition( + case r @ DropPartitions( ResolvedTable(_, _, table: SupportsPartitionManagement, _), partSpecs, _, _) => val partitionSchema = table.partitionSchema() r.copy(parts = resolvePartitionSpecs( @@ -51,7 +51,7 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { partitionSchema, requireExactMatchedPartitionSpec(table.name, _, partitionSchema.fieldNames))) - case r @ AlterTableRenamePartition( + case r @ RenamePartitions( ResolvedTable(_, _, table: SupportsPartitionManagement, _), from, to) => val partitionSchema = table.partitionSchema() val Seq(resolvedFrom, resolvedTo) = resolvePartitionSpecs( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala index 2fe6e20614524..12b1502dd7c38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala @@ -20,17 +20,17 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.JavaConverters._ import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnsetTableProperties} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper import org.apache.spark.sql.connector.catalog.TableCatalog /** - * A rule for resolving AlterTableUnsetProperties to handle non-existent properties. + * A rule for resolving [[UnsetTableProperties]] to handle non-existent properties. */ object ResolveTableProperties extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists => + case a @ UnsetTableProperties(r: ResolvedTable, props, ifExists) if !ifExists => val tblProperties = r.table.properties.asScala props.foreach { p => if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) { 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 8bb73702365b0..595a3a5ba5332 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 @@ -2767,7 +2767,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterNamespaceSetProperties]] logical plan. + * Create an [[SetNamespaceProperties]] logical plan. * * For example: * {{{ @@ -2778,14 +2778,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg override def visitSetNamespaceProperties(ctx: SetNamespacePropertiesContext): LogicalPlan = { withOrigin(ctx) { val properties = cleanNamespaceProperties(visitPropertyKeyValues(ctx.tablePropertyList), ctx) - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)), properties) } } /** - * Create an [[AlterNamespaceSetLocation]] logical plan. + * Create an [[SetNamespaceLocation]] logical plan. * * For example: * {{{ @@ -2794,7 +2794,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitSetNamespaceLocation(ctx: SetNamespaceLocationContext): LogicalPlan = { withOrigin(ctx) { - AlterNamespaceSetLocation( + SetNamespaceLocation( UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)), visitLocationSpec(ctx.locationSpec)) } @@ -3477,7 +3477,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Parse [[AlterViewSetProperties]] or [[AlterTableSetProperties]] commands. + * Parse [[SetViewProperties]] or [[SetTableProperties]] commands. * * For example: * {{{ @@ -3490,7 +3490,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val properties = visitPropertyKeyValues(ctx.tablePropertyList) val cleanedTableProperties = cleanTableProperties(ctx, properties) if (ctx.VIEW != null) { - AlterViewSetProperties( + SetViewProperties( createUnresolvedView( ctx.multipartIdentifier, commandName = "ALTER VIEW ... SET TBLPROPERTIES", @@ -3498,7 +3498,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg relationTypeMismatchHint = alterViewTypeMismatchHint), cleanedTableProperties) } else { - AlterTableSetProperties( + SetTableProperties( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... SET TBLPROPERTIES", @@ -3508,7 +3508,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Parse [[AlterViewUnsetProperties]] or [[AlterTableUnsetProperties]] commands. + * Parse [[UnsetViewProperties]] or [[UnsetTableProperties]] commands. * * For example: * {{{ @@ -3523,7 +3523,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val ifExists = ctx.EXISTS != null if (ctx.VIEW != null) { - AlterViewUnsetProperties( + UnsetViewProperties( createUnresolvedView( ctx.multipartIdentifier, commandName = "ALTER VIEW ... UNSET TBLPROPERTIES", @@ -3532,7 +3532,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg cleanedProperties, ifExists) } else { - AlterTableUnsetProperties( + UnsetTableProperties( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... UNSET TBLPROPERTIES", @@ -3543,7 +3543,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableSetLocation]] command. + * Create an [[SetTableLocation]] command. * * For example: * {{{ @@ -3551,7 +3551,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg * }}} */ override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetLocation( + SetTableLocation( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... SET LOCATION ...", @@ -3810,7 +3810,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableRecoverPartitions]] + * Create an [[RecoverPartitions]] * * For example: * {{{ @@ -3819,7 +3819,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitRecoverPartitions( ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) { - AlterTableRecoverPartitions( + RecoverPartitions( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... RECOVER PARTITIONS", @@ -3827,7 +3827,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableAddPartition]]. + * Create an [[AddPartitions]]. * * For example: * {{{ @@ -3849,7 +3849,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val location = Option(splCtx.locationSpec).map(visitLocationSpec) UnresolvedPartitionSpec(spec, location) } - AlterTableAddPartition( + AddPartitions( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... ADD PARTITION ...", @@ -3859,7 +3859,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableRenamePartition]] + * Create an [[RenamePartitions]] * * For example: * {{{ @@ -3868,7 +3868,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitRenameTablePartition( ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenamePartition( + RenamePartitions( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... RENAME TO PARTITION", @@ -3878,7 +3878,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableDropPartition]] + * Create an [[DropPartitions]] * * For example: * {{{ @@ -3897,7 +3897,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } val partSpecs = ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec) .map(spec => UnresolvedPartitionSpec(spec)) - AlterTableDropPartition( + DropPartitions( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... DROP PARTITION ...", @@ -3908,7 +3908,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } /** - * Create an [[AlterTableSerDeProperties]] + * Create an [[SetTableSerDeProperties]] * * For example: * {{{ @@ -3918,7 +3918,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg * }}} */ override def visitSetTableSerDe(ctx: SetTableSerDeContext): LogicalPlan = withOrigin(ctx) { - AlterTableSerDeProperties( + SetTableSerDeProperties( createUnresolvedTable( ctx.multipartIdentifier, "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", 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 f9341714881fa..8797b107f945a 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 @@ -305,7 +305,7 @@ case class DescribeNamespace( * The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET (DBPROPERTIES|PROPERTIES) * command. */ -case class AlterNamespaceSetProperties( +case class SetNamespaceProperties( namespace: LogicalPlan, properties: Map[String, String]) extends Command { override def children: Seq[LogicalPlan] = Seq(namespace) @@ -314,7 +314,7 @@ case class AlterNamespaceSetProperties( /** * The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION command. */ -case class AlterNamespaceSetLocation( +case class SetNamespaceLocation( namespace: LogicalPlan, location: String) extends Command { override def children: Seq[LogicalPlan] = Seq(namespace) @@ -676,7 +676,7 @@ case class AnalyzeColumn( * PARTITION spec1 [LOCATION 'loc1'][, PARTITION spec2 [LOCATION 'loc2'], ...]; * }}} */ -case class AlterTableAddPartition( +case class AddPartitions( child: LogicalPlan, parts: Seq[PartitionSpec], ifNotExists: Boolean) extends Command { @@ -698,7 +698,7 @@ case class AlterTableAddPartition( * ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; * }}} */ -case class AlterTableDropPartition( +case class DropPartitions( child: LogicalPlan, parts: Seq[PartitionSpec], ifExists: Boolean, @@ -712,7 +712,7 @@ case class AlterTableDropPartition( /** * The logical plan of the ALTER TABLE ... RENAME TO PARTITION command. */ -case class AlterTableRenamePartition( +case class RenamePartitions( child: LogicalPlan, from: PartitionSpec, to: PartitionSpec) extends Command { @@ -727,7 +727,7 @@ case class AlterTableRenamePartition( /** * The logical plan of the ALTER TABLE ... RECOVER PARTITIONS command. */ -case class AlterTableRecoverPartitions(child: LogicalPlan) extends Command { +case class RecoverPartitions(child: LogicalPlan) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } @@ -819,7 +819,7 @@ case class AlterViewAs( /** * The logical plan of the ALTER VIEW ... SET TBLPROPERTIES command. */ -case class AlterViewSetProperties( +case class SetViewProperties( child: LogicalPlan, properties: Map[String, String]) extends Command { override def children: Seq[LogicalPlan] = child :: Nil @@ -828,7 +828,7 @@ case class AlterViewSetProperties( /** * The logical plan of the ALTER VIEW ... UNSET TBLPROPERTIES command. */ -case class AlterViewUnsetProperties( +case class UnsetViewProperties( child: LogicalPlan, propertyKeys: Seq[String], ifExists: Boolean) extends Command { @@ -838,7 +838,7 @@ case class AlterViewUnsetProperties( /** * The logical plan of the ALTER TABLE ... SET [SERDE|SERDEPROPERTIES] command. */ -case class AlterTableSerDeProperties( +case class SetTableSerDeProperties( child: LogicalPlan, serdeClassName: Option[String], serdeProperties: Option[Map[String, String]], @@ -876,7 +876,7 @@ case class UncacheTable( /** * The logical plan of the ALTER TABLE ... SET LOCATION command. */ -case class AlterTableSetLocation( +case class SetTableLocation( table: LogicalPlan, partitionSpec: Option[TablePartitionSpec], location: String) extends Command { @@ -886,7 +886,7 @@ case class AlterTableSetLocation( /** * The logical plan of the ALTER TABLE ... SET TBLPROPERTIES command. */ -case class AlterTableSetProperties( +case class SetTableProperties( table: LogicalPlan, properties: Map[String, String]) extends Command { override def children: Seq[LogicalPlan] = table :: Nil @@ -895,7 +895,7 @@ case class AlterTableSetProperties( /** * The logical plan of the ALTER TABLE ... UNSET TBLPROPERTIES command. */ -case class AlterTableUnsetProperties( +case class UnsetTableProperties( table: LogicalPlan, propertyKeys: Seq[String], ifExists: Boolean) extends 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 95921bbd129d0..c5c2451625124 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 @@ -741,16 +741,16 @@ class DDLParserSuite extends AnalysisTest { val hint = Some("Please use ALTER TABLE instead.") comparePlans(parsePlan(sql1_view), - AlterViewSetProperties( + SetViewProperties( UnresolvedView(Seq("table_name"), "ALTER VIEW ... SET TBLPROPERTIES", false, hint), Map("test" -> "test", "comment" -> "new_comment"))) comparePlans(parsePlan(sql2_view), - AlterViewUnsetProperties( + UnsetViewProperties( UnresolvedView(Seq("table_name"), "ALTER VIEW ... UNSET TBLPROPERTIES", false, hint), Seq("comment", "test"), ifExists = false)) comparePlans(parsePlan(sql3_view), - AlterViewUnsetProperties( + UnsetViewProperties( UnresolvedView(Seq("table_name"), "ALTER VIEW ... UNSET TBLPROPERTIES", false, hint), Seq("comment", "test"), ifExists = true)) @@ -767,18 +767,18 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql1_table), - AlterTableSetProperties( + SetTableProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET TBLPROPERTIES", hint), Map("test" -> "test", "comment" -> "new_comment"))) comparePlans( parsePlan(sql2_table), - AlterTableUnsetProperties( + UnsetTableProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", hint), Seq("comment", "test"), ifExists = false)) comparePlans( parsePlan(sql3_table), - AlterTableUnsetProperties( + UnsetTableProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", hint), Seq("comment", "test"), ifExists = true)) @@ -876,14 +876,14 @@ class DDLParserSuite extends AnalysisTest { val hint = Some("Please use ALTER VIEW instead.") comparePlans( parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"), - AlterTableSetLocation( + SetTableLocation( UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", hint), None, "new location")) comparePlans( parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"), - AlterTableSetLocation( + SetTableLocation( UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", hint), Some(Map("ds" -> "2017-06-10")), "new location")) @@ -1771,49 +1771,49 @@ class DDLParserSuite extends AnalysisTest { test("set namespace properties") { comparePlans( parsePlan("ALTER DATABASE a.b.c SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')"), - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a", "b" -> "b", "c" -> "c"))) comparePlans( parsePlan("ALTER SCHEMA a.b.c SET PROPERTIES ('a'='a')"), - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a"))) comparePlans( parsePlan("ALTER NAMESPACE a.b.c SET PROPERTIES ('b'='b')"), - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(Seq("a", "b", "c")), Map("b" -> "b"))) comparePlans( parsePlan("ALTER DATABASE a.b.c SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')"), - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a", "b" -> "b", "c" -> "c"))) comparePlans( parsePlan("ALTER SCHEMA a.b.c SET DBPROPERTIES ('a'='a')"), - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a"))) comparePlans( parsePlan("ALTER NAMESPACE a.b.c SET DBPROPERTIES ('b'='b')"), - AlterNamespaceSetProperties( + SetNamespaceProperties( UnresolvedNamespace(Seq("a", "b", "c")), Map("b" -> "b"))) } test("set namespace location") { comparePlans( parsePlan("ALTER DATABASE a.b.c SET LOCATION '/home/user/db'"), - AlterNamespaceSetLocation( + SetNamespaceLocation( UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db")) comparePlans( parsePlan("ALTER SCHEMA a.b.c SET LOCATION '/home/user/db'"), - AlterNamespaceSetLocation( + SetNamespaceLocation( UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db")) comparePlans( parsePlan("ALTER NAMESPACE a.b.c SET LOCATION '/home/user/db'"), - AlterNamespaceSetLocation( + SetNamespaceLocation( UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db")) } @@ -2070,7 +2070,7 @@ class DDLParserSuite extends AnalysisTest { val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" val hint = Some("Please use ALTER VIEW instead.") val parsed1 = parsePlan(sql1) - val expected1 = AlterTableSerDeProperties( + val expected1 = SetTableSerDeProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), None, @@ -2083,7 +2083,7 @@ class DDLParserSuite extends AnalysisTest { |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin val parsed2 = parsePlan(sql2) - val expected2 = AlterTableSerDeProperties( + val expected2 = SetTableSerDeProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -2096,7 +2096,7 @@ class DDLParserSuite extends AnalysisTest { |SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin val parsed3 = parsePlan(sql3) - val expected3 = AlterTableSerDeProperties( + val expected3 = SetTableSerDeProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -2110,7 +2110,7 @@ class DDLParserSuite extends AnalysisTest { |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin val parsed4 = parsePlan(sql4) - val expected4 = AlterTableSerDeProperties( + val expected4 = SetTableSerDeProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -2123,7 +2123,7 @@ class DDLParserSuite extends AnalysisTest { |SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin val parsed5 = parsePlan(sql5) - val expected5 = AlterTableSerDeProperties( + val expected5 = SetTableSerDeProperties( UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -2136,7 +2136,7 @@ class DDLParserSuite extends AnalysisTest { |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin val parsed6 = parsePlan(sql6) - val expected6 = AlterTableSerDeProperties( + val expected6 = SetTableSerDeProperties( UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -2149,7 +2149,7 @@ class DDLParserSuite extends AnalysisTest { |SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin val parsed7 = parsePlan(sql7) - val expected7 = AlterTableSerDeProperties( + val expected7 = SetTableSerDeProperties( UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), 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 4d974b4515bcb..7a8f4dd39080e 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 @@ -167,25 +167,25 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) createAlterTable(nameParts, catalog, tbl, changes) } - case AlterTableSetProperties(ResolvedV1TableIdentifier(ident), props) => + case SetTableProperties(ResolvedV1TableIdentifier(ident), props) => AlterTableSetPropertiesCommand(ident.asTableIdentifier, props, isView = false) - case AlterTableUnsetProperties(ResolvedV1TableIdentifier(ident), keys, ifExists) => + case UnsetTableProperties(ResolvedV1TableIdentifier(ident), keys, ifExists) => AlterTableUnsetPropertiesCommand(ident.asTableIdentifier, keys, ifExists, isView = false) - case AlterViewSetProperties(ResolvedView(ident, _), props) => + case SetViewProperties(ResolvedView(ident, _), props) => AlterTableSetPropertiesCommand(ident.asTableIdentifier, props, isView = true) - case AlterViewUnsetProperties(ResolvedView(ident, _), keys, ifExists) => + case UnsetViewProperties(ResolvedView(ident, _), keys, ifExists) => AlterTableUnsetPropertiesCommand(ident.asTableIdentifier, keys, ifExists, isView = true) case d @ DescribeNamespace(DatabaseInSessionCatalog(db), _) => DescribeDatabaseCommand(db, d.extended) - case AlterNamespaceSetProperties(DatabaseInSessionCatalog(db), properties) => + case SetNamespaceProperties(DatabaseInSessionCatalog(db), properties) => AlterDatabasePropertiesCommand(db, properties) - case AlterNamespaceSetLocation(DatabaseInSessionCatalog(db), location) => + case SetNamespaceLocation(DatabaseInSessionCatalog(db), location) => AlterDatabaseSetLocationCommand(db, location) case s @ ShowNamespaces(ResolvedNamespace(cata, _), _, output) if isSessionCatalog(cata) => @@ -417,24 +417,24 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } ShowColumnsCommand(db, v1TableName, output) - case AlterTableRecoverPartitions(ResolvedV1TableIdentifier(ident)) => + case RecoverPartitions(ResolvedV1TableIdentifier(ident)) => AlterTableRecoverPartitionsCommand( ident.asTableIdentifier, "ALTER TABLE RECOVER PARTITIONS") - case AlterTableAddPartition(ResolvedV1TableIdentifier(ident), partSpecsAndLocs, ifNotExists) => + case AddPartitions(ResolvedV1TableIdentifier(ident), partSpecsAndLocs, ifNotExists) => AlterTableAddPartitionCommand( ident.asTableIdentifier, partSpecsAndLocs.asUnresolvedPartitionSpecs.map(spec => (spec.spec, spec.location)), ifNotExists) - case AlterTableRenamePartition( + case RenamePartitions( ResolvedV1TableIdentifier(ident), UnresolvedPartitionSpec(from, _), UnresolvedPartitionSpec(to, _)) => AlterTableRenamePartitionCommand(ident.asTableIdentifier, from, to) - case AlterTableDropPartition( + case DropPartitions( ResolvedV1TableIdentifier(ident), specs, ifExists, purge) => AlterTableDropPartitionCommand( ident.asTableIdentifier, @@ -443,7 +443,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) purge, retainData = false) - case AlterTableSerDeProperties( + case SetTableSerDeProperties( ResolvedV1TableIdentifier(ident), serdeClassName, serdeProperties, @@ -454,7 +454,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) serdeProperties, partitionSpec) - case AlterTableSetLocation(ResolvedV1TableIdentifier(ident), partitionSpec, location) => + case SetTableLocation(ResolvedV1TableIdentifier(ident), partitionSpec, location) => AlterTableSetLocationCommand(ident.asTableIdentifier, partitionSpec, location) case AlterViewAs(ResolvedView(ident, _), originalText, query) => 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 976c7df841dd9..c633442d7b2aa 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 @@ -306,10 +306,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat invalidateTableCache(r), session.sharedState.cacheManager.cacheQuery) :: Nil - case AlterNamespaceSetProperties(ResolvedNamespace(catalog, ns), properties) => + case SetNamespaceProperties(ResolvedNamespace(catalog, ns), properties) => AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil - case AlterNamespaceSetLocation(ResolvedNamespace(catalog, ns), location) => + case SetNamespaceLocation(ResolvedNamespace(catalog, ns), location) => AlterNamespaceSetPropertiesExec( catalog.asNamespaceCatalog, ns, @@ -352,7 +352,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case AnalyzeTable(_: ResolvedTable, _, _) | AnalyzeColumn(_: ResolvedTable, _, _) => throw new AnalysisException("ANALYZE TABLE is not supported for v2 tables.") - case AlterTableAddPartition( + case AddPartitions( r @ ResolvedTable(_, _, table: SupportsPartitionManagement, _), parts, ignoreIfExists) => AddPartitionExec( table, @@ -360,7 +360,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat ignoreIfExists, recacheTable(r)) :: Nil - case AlterTableDropPartition( + case DropPartitions( r @ ResolvedTable(_, _, table: SupportsPartitionManagement, _), parts, ignoreIfNotExists, @@ -372,7 +372,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat purge, recacheTable(r)) :: Nil - case AlterTableRenamePartition( + case RenamePartitions( r @ ResolvedTable(_, _, table: SupportsPartitionManagement, _), from, to) => RenamePartitionExec( table, @@ -380,11 +380,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat Seq(to).asResolvedPartitionSpecs.head, recacheTable(r)) :: Nil - case AlterTableRecoverPartitions(_: ResolvedTable) => + case RecoverPartitions(_: ResolvedTable) => throw new AnalysisException( "ALTER TABLE ... RECOVER PARTITIONS is not supported for v2 tables.") - case AlterTableSerDeProperties(_: ResolvedTable, _, _, _) => + case SetTableSerDeProperties(_: ResolvedTable, _, _, _) => throw new AnalysisException( "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES] is not supported for v2 tables.") @@ -421,20 +421,20 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case r: UncacheTable => UncacheTableExec(r.table, cascade = !r.isTempView) :: Nil - case AlterTableSetLocation(table: ResolvedTable, partitionSpec, location) => + case SetTableLocation(table: ResolvedTable, partitionSpec, location) => if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.alterV2TableSetLocationWithPartitionNotSupportedError } val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, location)) AlterTableExec(table.catalog, table.identifier, changes) :: Nil - case AlterTableSetProperties(table: ResolvedTable, props) => + case SetTableProperties(table: ResolvedTable, props) => val changes = props.map { case (key, value) => TableChange.setProperty(key, value) }.toSeq AlterTableExec(table.catalog, table.identifier, changes) :: Nil - case AlterTableUnsetProperties(table: ResolvedTable, keys, _) => + case UnsetTableProperties(table: ResolvedTable, keys, _) => val changes = keys.map(key => TableChange.removeProperty(key)) AlterTableExec(table.catalog, table.identifier, changes) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 145daaffd035f..33f70f06aa209 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, FunctionIdenti import org.apache.spark.sql.catalyst.analysis.UnresolvedTable import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.plans.logical.{AlterTableRecoverPartitions, LocalRelation, LogicalPlan, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, RecoverPartitions, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.types.StructType @@ -448,7 +448,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { override def recoverPartitions(tableName: String): Unit = { val multiPartIdent = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(tableName) sparkSession.sessionState.executePlan( - AlterTableRecoverPartitions( + RecoverPartitions( UnresolvedTable(multiPartIdent, "recoverPartitions()", None))).toRdd } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 7d67e1cb72815..b1a3959af036b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1241,7 +1241,7 @@ class DataSourceV2SQLSuite } } - test("AlterNamespaceSetProperties using v2 catalog") { + test("ALTER NAMESPACE .. SET PROPERTIES using v2 catalog") { withNamespace("testcat.ns1.ns2") { sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + "'test namespace' LOCATION '/tmp/ns_test' WITH PROPERTIES ('a'='a','b'='b','c'='c')") @@ -1257,7 +1257,7 @@ class DataSourceV2SQLSuite } } - test("AlterNamespaceSetProperties: reserved properties") { + test("ALTER NAMESPACE .. SET PROPERTIES reserved properties") { import SupportsNamespaces._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => @@ -1288,7 +1288,7 @@ class DataSourceV2SQLSuite } } - test("AlterNamespaceSetLocation using v2 catalog") { + test("ALTER NAMESPACE .. SET LOCATION using v2 catalog") { withNamespace("testcat.ns1.ns2") { sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + "'test namespace' LOCATION '/tmp/ns_test_1'") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala index 1ec0f45f66118..1694c73b10f2c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedPartitionSpec, UnresolvedTable} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan -import org.apache.spark.sql.catalyst.plans.logical.AlterTableAddPartition +import org.apache.spark.sql.catalyst.plans.logical.AddPartitions import org.apache.spark.sql.test.SharedSparkSession class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSession { @@ -29,7 +29,7 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes |(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION |(dt='2009-09-09', country='uk')""".stripMargin val parsed = parsePlan(sql) - val expected = AlterTableAddPartition( + val expected = AddPartitions( UnresolvedTable( Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ...", @@ -44,7 +44,7 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes test("add partition") { val sql = "ALTER TABLE a.b.c ADD PARTITION (dt='2008-08-08') LOCATION 'loc'" val parsed = parsePlan(sql) - val expected = AlterTableAddPartition( + val expected = AddPartitions( UnresolvedTable( Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ...", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala index b48ca16a6bb45..4c60c80f4e054 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedPartitionSpec, UnresolvedTable} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.AlterTableDropPartition +import org.apache.spark.sql.catalyst.plans.logical.DropPartitions import org.apache.spark.sql.test.SharedSparkSession class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSession { @@ -29,7 +29,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe |ALTER TABLE table_name DROP PARTITION |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') """.stripMargin - val expected = AlterTableDropPartition( + val expected = DropPartitions( UnresolvedTable( Seq("table_name"), "ALTER TABLE ... DROP PARTITION ...", @@ -49,7 +49,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe |PARTITION (dt='2008-08-08', country='us'), |PARTITION (dt='2009-09-09', country='uk') """.stripMargin - val expected = AlterTableDropPartition( + val expected = DropPartitions( UnresolvedTable( Seq("table_name"), "ALTER TABLE ... DROP PARTITION ...", @@ -64,7 +64,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe test("drop partition in a table with multi-part identifier") { val sql = "ALTER TABLE a.b.c DROP IF EXISTS PARTITION (ds='2017-06-10')" - val expected = AlterTableDropPartition( + val expected = DropPartitions( UnresolvedTable( Seq("a", "b", "c"), "ALTER TABLE ... DROP PARTITION ...", @@ -78,7 +78,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe test("drop partition with PURGE") { val sql = "ALTER TABLE table_name DROP PARTITION (p=1) PURGE" - val expected = AlterTableDropPartition( + val expected = DropPartitions( UnresolvedTable( Seq("table_name"), "ALTER TABLE ... DROP PARTITION ...", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala index 04251b665c05e..ebc1bd3468837 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedTable} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.AlterTableRecoverPartitions +import org.apache.spark.sql.catalyst.plans.logical.RecoverPartitions import org.apache.spark.sql.test.SharedSparkSession class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSparkSession { @@ -35,7 +35,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa test("recover partitions of a table") { comparePlans( parsePlan("ALTER TABLE tbl RECOVER PARTITIONS"), - AlterTableRecoverPartitions( + RecoverPartitions( UnresolvedTable( Seq("tbl"), "ALTER TABLE ... RECOVER PARTITIONS", @@ -45,7 +45,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa test("recover partitions of a table in a database") { comparePlans( parsePlan("alter table db.tbl recover partitions"), - AlterTableRecoverPartitions( + RecoverPartitions( UnresolvedTable( Seq("db", "tbl"), "ALTER TABLE ... RECOVER PARTITIONS", @@ -55,7 +55,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa test("recover partitions of a table spark_catalog") { comparePlans( parsePlan("alter table spark_catalog.db.TBL recover partitions"), - AlterTableRecoverPartitions( + RecoverPartitions( UnresolvedTable( Seq("spark_catalog", "db", "TBL"), "ALTER TABLE ... RECOVER PARTITIONS", @@ -65,7 +65,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa test("recover partitions of a table in nested namespaces") { comparePlans( parsePlan("Alter Table ns1.ns2.ns3.ns4.ns5.ns6.ns7.ns8.t Recover Partitions"), - AlterTableRecoverPartitions( + RecoverPartitions( UnresolvedTable( Seq("ns1", "ns2", "ns3", "ns4", "ns5", "ns6", "ns7", "ns8", "t"), "ALTER TABLE ... RECOVER PARTITIONS", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala index 5f2856f071df7..4148798d6cdb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedPartitionSpec, UnresolvedTable} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan -import org.apache.spark.sql.catalyst.plans.logical.AlterTableRenamePartition +import org.apache.spark.sql.catalyst.plans.logical.RenamePartitions import org.apache.spark.sql.test.SharedSparkSession class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSparkSession { @@ -29,7 +29,7 @@ class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSpark |RENAME TO PARTITION (ds='2018-06-10') """.stripMargin val parsed = parsePlan(sql) - val expected = AlterTableRenamePartition( + val expected = RenamePartitions( UnresolvedTable( Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO PARTITION", @@ -45,7 +45,7 @@ class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSpark |RENAME TO PARTITION (dt='2008-09-09', country='uk') """.stripMargin val parsed = parsePlan(sql) - val expected = AlterTableRenamePartition( + val expected = RenamePartitions( UnresolvedTable( Seq("table_name"), "ALTER TABLE ... RENAME TO PARTITION", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 848f15c0a6a9e..17c44bc9ac768 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, EmptyFunc import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AlterTableSetLocation, AlterTableSetProperties, AlterTableUnsetProperties, AppendData, Assignment, CreateTableAsSelect, CreateTableStatement, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, ShowTableProperties, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, Assignment, CreateTableAsSelect, CreateTableStatement, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCapability, TableCatalog, TableChange, V1Table} @@ -780,23 +780,23 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed3, expected3) } else { parsed1 match { - case AlterTableSetProperties(_: ResolvedTable, properties) => + case SetTableProperties(_: ResolvedTable, properties) => assert(properties == Map(("test", "test"), ("comment", "new_comment"))) - case _ => fail("expect AlterTableSetProperties") + case _ => fail(s"expect ${SetTableProperties.getClass.getName}") } parsed2 match { - case AlterTableUnsetProperties(_: ResolvedTable, propertyKeys, ifExists) => + case UnsetTableProperties(_: ResolvedTable, propertyKeys, ifExists) => assert(propertyKeys == Seq("comment", "test")) assert(!ifExists) - case _ => fail("expect AlterTableUnsetProperties") + case _ => fail(s"expect ${UnsetTableProperties.getClass.getName}") } parsed3 match { - case AlterTableUnsetProperties(_: ResolvedTable, propertyKeys, ifExists) => + case UnsetTableProperties(_: ResolvedTable, propertyKeys, ifExists) => assert(propertyKeys == Seq("comment", "test")) assert(ifExists) - case _ => fail("expect AlterTableUnsetProperties") + case _ => fail(s"expect ${UnsetTableProperties.getClass.getName}") } } } @@ -808,12 +808,14 @@ class PlanResolutionSuite extends AnalysisTest { // For non-existing tables, we convert it to v2 command with `UnresolvedV2Table` parsed4 match { - case AlterTableSetProperties(_: UnresolvedTable, _) => // OK - case _ => fail("Expect AlterTableSetProperties, but got:\n" + parsed4.treeString) + case SetTableProperties(_: UnresolvedTable, _) => // OK + case _ => + fail(s"Expect ${SetTableProperties.getClass.getName}, but got:\n" + parsed4.treeString) } parsed5 match { - case AlterTableUnsetProperties(_: UnresolvedTable, _, _) => // OK - case _ => fail("Expect AlterTableUnsetProperties, but got:\n" + parsed5.treeString) + case UnsetTableProperties(_: UnresolvedTable, _, _) => // OK + case _ => + fail(s"Expect ${UnsetTableProperties.getClass.getName}, but got:\n" + parsed5.treeString) } } @@ -835,7 +837,7 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed, expected) } else { parsed match { - case AlterTableSetProperties(_: ResolvedTable, changes) => + case SetTableProperties(_: ResolvedTable, changes) => assert(changes == Map(("a", "1"), ("b", "0.1"), ("c", "true"))) case _ => fail("Expect AlterTable, but got:\n" + parsed.treeString) } @@ -856,9 +858,10 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed, expected) } else { parsed match { - case AlterTableSetLocation(_: ResolvedTable, _, location) => + case SetTableLocation(_: ResolvedTable, _, location) => assert(location === "new location") - case _ => fail("Expect AlterTableSetLocation, but got:\n" + parsed.treeString) + case _ => + fail(s"Expect ${SetTableLocation.getClass.getName}, but got:\n" + parsed.treeString) } } }