Skip to content

Commit

Permalink
[SPARK-32512][SQL] add alter table add/drop partition command for dat…
Browse files Browse the repository at this point in the history
…asourcev2

### What changes were proposed in this pull request?
This patch is trying to add `AlterTableAddPartitionExec` and `AlterTableDropPartitionExec` with the new table partition API, defined in #28617.

### Does this PR introduce _any_ user-facing change?
Yes. User can use `alter table add partition` or `alter table drop partition` to create/drop partition in V2Table.

### How was this patch tested?
Run suites and fix old tests.

Closes #29339 from stczwd/SPARK-32512-new.

Lead-authored-by: stczwd <qcsd2011@163.com>
Co-authored-by: Jacky Lee <qcsd2011@163.com>
Co-authored-by: Jackey Lee <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
jackylee-ch authored and cloud-fan committed Nov 11, 2020
1 parent 8760032 commit 1eb236b
Show file tree
Hide file tree
Showing 19 changed files with 670 additions and 97 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -225,6 +225,7 @@ class Analyzer(
ResolveInsertInto ::
ResolveRelations ::
ResolveTables ::
ResolvePartitionSpec ::
ResolveReferences ::
ResolveCreateNamedStruct ::
ResolveDeserializer ::
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.connector.catalog.{SupportsAtomicPartitionManagement, SupportsPartitionManagement, Table}
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, After, ColumnPosition, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnPosition, UpdateColumnType}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -564,6 +565,12 @@ trait CheckAnalysis extends PredicateHelper {
// no validation needed for set and remove property
}

case AlterTableAddPartition(ResolvedTable(_, _, table), parts, _) =>
checkAlterTablePartition(table, parts)

case AlterTableDropPartition(ResolvedTable(_, _, table), parts, _, _, _) =>
checkAlterTablePartition(table, parts)

case _ => // Fallbacks to the following checks
}

Expand Down Expand Up @@ -976,4 +983,24 @@ trait CheckAnalysis extends PredicateHelper {
failOnOuterReferenceInSubTree(p)
}}
}

// Make sure that table is able to alter partition.
private def checkAlterTablePartition(
table: Table, parts: Seq[PartitionSpec]): Unit = {
(table, parts) match {
case (_, parts) if parts.exists(_.isInstanceOf[UnresolvedPartitionSpec]) =>
failAnalysis("PartitionSpecs are not resolved")

case (table, _) if !table.isInstanceOf[SupportsPartitionManagement] =>
failAnalysis(s"Table ${table.name()} can not alter partitions.")

// Skip atomic partition tables
case (_: SupportsAtomicPartitionManagement, _) =>
case (_: SupportsPartitionManagement, parts) if parts.size > 1 =>
failAnalysis(
s"Nonatomic partition table ${table.name()} can not alter multiple partitions.")

case _ =>
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement
import org.apache.spark.sql.types._

/**
* Resolve [[UnresolvedPartitionSpec]] to [[ResolvedPartitionSpec]] in partition related commands.
*/
object ResolvePartitionSpec extends Rule[LogicalPlan] {

def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case r @ AlterTableAddPartition(
ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs, _) =>
r.copy(parts = resolvePartitionSpecs(partSpecs, table.partitionSchema()))

case r @ AlterTableDropPartition(
ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs, _, _, _) =>
r.copy(parts = resolvePartitionSpecs(partSpecs, table.partitionSchema()))
}

private def resolvePartitionSpecs(
partSpecs: Seq[PartitionSpec], partSchema: StructType): Seq[ResolvedPartitionSpec] =
partSpecs.map {
case unresolvedPartSpec: UnresolvedPartitionSpec =>
ResolvedPartitionSpec(
convertToPartIdent(unresolvedPartSpec.spec, partSchema), unresolvedPartSpec.location)
case resolvedPartitionSpec: ResolvedPartitionSpec =>
resolvedPartitionSpec
}

private def convertToPartIdent(
partSpec: TablePartitionSpec, partSchema: StructType): InternalRow = {
val conflictKeys = partSpec.keys.toSeq.diff(partSchema.map(_.name))
if (conflictKeys.nonEmpty) {
throw new AnalysisException(s"Partition key ${conflictKeys.mkString(",")} not exists")
}

val partValues = partSchema.map { part =>
val partValue = partSpec.get(part.name).orNull
if (partValue == null) {
null
} else {
// TODO: Support other datatypes, such as DateType
part.dataType match {
case _: ByteType =>
partValue.toByte
case _: ShortType =>
partValue.toShort
case _: IntegerType =>
partValue.toInt
case _: LongType =>
partValue.toLong
case _: FloatType =>
partValue.toFloat
case _: DoubleType =>
partValue.toDouble
case _: StringType =>
partValue
case _ =>
throw new AnalysisException(
s"Type ${part.dataType.typeName} is not supported for partition.")
}
}
}
InternalRow.fromSeq(partValues)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@

package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.catalog.CatalogFunction
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}
import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsNamespaces, Table, TableCatalog}
Expand Down Expand Up @@ -53,6 +55,12 @@ case class UnresolvedTableOrView(
override def output: Seq[Attribute] = Nil
}

sealed trait PartitionSpec

case class UnresolvedPartitionSpec(
spec: TablePartitionSpec,
location: Option[String] = None) extends PartitionSpec

/**
* Holds the name of a function that has yet to be looked up in a catalog. It will be resolved to
* [[ResolvedFunc]] during analysis.
Expand All @@ -78,6 +86,10 @@ case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: T
override def output: Seq[Attribute] = Nil
}

case class ResolvedPartitionSpec(
spec: InternalRow,
location: Option[String] = None) extends PartitionSpec

/**
* A plan containing resolved (temp) views.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3415,7 +3415,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

/**
* Create an [[AlterTableAddPartitionStatement]].
* Create an [[AlterTableAddPartition]].
*
* For example:
* {{{
Expand All @@ -3435,10 +3435,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
val specsAndLocs = ctx.partitionSpecLocation.asScala.map { splCtx =>
val spec = visitNonOptionalPartitionSpec(splCtx.partitionSpec)
val location = Option(splCtx.locationSpec).map(visitLocationSpec)
spec -> location
UnresolvedPartitionSpec(spec, location)
}
AlterTableAddPartitionStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
AlterTableAddPartition(
UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)),
specsAndLocs.toSeq,
ctx.EXISTS != null)
}
Expand All @@ -3460,7 +3460,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

/**
* Create an [[AlterTableDropPartitionStatement]]
* Create an [[AlterTableDropPartition]]
*
* For example:
* {{{
Expand All @@ -3477,9 +3477,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
if (ctx.VIEW != null) {
operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx)
}
AlterTableDropPartitionStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec).toSeq,
val partSpecs = ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec)
.map(spec => UnresolvedPartitionSpec(spec))
AlterTableDropPartition(
UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)),
partSpecs.toSeq,
ifExists = ctx.EXISTS != null,
purge = ctx.PURGE != null,
retainData = false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -217,14 +217,6 @@ case class AlterTableSetLocationStatement(
case class AlterTableRecoverPartitionsStatement(
tableName: Seq[String]) extends ParsedStatement

/**
* ALTER TABLE ... ADD PARTITION command, as parsed from SQL
*/
case class AlterTableAddPartitionStatement(
tableName: Seq[String],
partitionSpecsAndLocs: Seq[(TablePartitionSpec, Option[String])],
ifNotExists: Boolean) extends ParsedStatement

/**
* ALTER TABLE ... RENAME PARTITION command, as parsed from SQL.
*/
Expand All @@ -233,16 +225,6 @@ case class AlterTableRenamePartitionStatement(
from: TablePartitionSpec,
to: TablePartitionSpec) extends ParsedStatement

/**
* ALTER TABLE ... DROP PARTITION command, as parsed from SQL
*/
case class AlterTableDropPartitionStatement(
tableName: Seq[String],
specs: Seq[TablePartitionSpec],
ifExists: Boolean,
purge: Boolean,
retainData: Boolean) extends ParsedStatement

/**
* ALTER TABLE ... SERDEPROPERTIES command, as parsed from SQL
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.catalyst.plans.logical

import org.apache.spark.sql.catalyst.analysis.{NamedRelation, UnresolvedException}
import org.apache.spark.sql.catalyst.analysis.{NamedRelation, PartitionSpec, ResolvedPartitionSpec, UnresolvedException}
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, Unevaluable}
import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema
Expand Down Expand Up @@ -612,6 +612,46 @@ case class AnalyzeColumn(
override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the ALTER TABLE ADD PARTITION command.
*
* The syntax of this command is:
* {{{
* ALTER TABLE table ADD [IF NOT EXISTS]
* PARTITION spec1 [LOCATION 'loc1'][, PARTITION spec2 [LOCATION 'loc2'], ...];
* }}}
*/
case class AlterTableAddPartition(
child: LogicalPlan,
parts: Seq[PartitionSpec],
ifNotExists: Boolean) extends Command {
override lazy val resolved: Boolean =
childrenResolved && parts.forall(_.isInstanceOf[ResolvedPartitionSpec])

override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the ALTER TABLE DROP PARTITION command.
* This may remove the data and metadata for this partition.
*
* The syntax of this command is:
* {{{
* ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...];
* }}}
*/
case class AlterTableDropPartition(
child: LogicalPlan,
parts: Seq[PartitionSpec],
ifExists: Boolean,
purge: Boolean,
retainData: Boolean) extends Command {
override lazy val resolved: Boolean =
childrenResolved && parts.forall(_.isInstanceOf[ResolvedPartitionSpec])

override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the LOAD DATA INTO TABLE command.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2
import scala.collection.JavaConverters._

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedPartitionSpec, UnresolvedPartitionSpec}
import org.apache.spark.sql.connector.catalog.{SupportsAtomicPartitionManagement, SupportsDelete, SupportsPartitionManagement, SupportsRead, SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.util.CaseInsensitiveStringMap

object DataSourceV2Implicits {
Expand Down Expand Up @@ -52,6 +53,26 @@ object DataSourceV2Implicits {
}
}

def asPartitionable: SupportsPartitionManagement = {
table match {
case support: SupportsPartitionManagement =>
support
case _ =>
throw new AnalysisException(
s"Table does not support partition management: ${table.name}")
}
}

def asAtomicPartitionable: SupportsAtomicPartitionManagement = {
table match {
case support: SupportsAtomicPartitionManagement =>
support
case _ =>
throw new AnalysisException(
s"Table does not support atomic partition management: ${table.name}")
}
}

def supports(capability: TableCapability): Boolean = table.capabilities.contains(capability)

def supportsAny(capabilities: TableCapability*): Boolean = capabilities.exists(supports)
Expand All @@ -62,4 +83,12 @@ object DataSourceV2Implicits {
new CaseInsensitiveStringMap(options.asJava)
}
}

implicit class PartitionSpecsHelper(partSpecs: Seq[PartitionSpec]) {
def asUnresolvedPartitionSpecs: Seq[UnresolvedPartitionSpec] =
partSpecs.map(_.asInstanceOf[UnresolvedPartitionSpec])

def asResolvedPartitionSpecs: Seq[ResolvedPartitionSpec] =
partSpecs.map(_.asInstanceOf[ResolvedPartitionSpec])
}
}
Loading

0 comments on commit 1eb236b

Please sign in to comment.