Skip to content

Commit 52506f1

Browse files
committed
Fix
1 parent 4e72f7e commit 52506f1

File tree

3 files changed

+29
-84
lines changed

3 files changed

+29
-84
lines changed

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

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -306,8 +306,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
306306
}
307307
// We cannot use UnresolvedAttribute because resolution is performed after Analysis, when
308308
// running the command. The type is not relevant, it is replaced during the real resolution
309-
val partition =
310-
AttributeReference(pFilter.identifier().getText, StringType)()
309+
val partition = UnresolvedAttribute(pFilter.identifier().getText)
311310
val value = Literal(visitStringConstant(pFilter.constant()))
312311
val operator = pFilter.comparisonOperator().getChild(0).asInstanceOf[TerminalNode]
313312
buildComparison(partition, value, operator)

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,13 @@ package org.apache.spark.sql.execution
2020
import java.util.Locale
2121

2222
import scala.collection.JavaConverters._
23-
2423
import org.antlr.v4.runtime.{ParserRuleContext, Token}
2524
import org.antlr.v4.runtime.tree.TerminalNode
26-
2725
import org.apache.spark.sql.SaveMode
2826
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
27+
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
2928
import org.apache.spark.sql.catalyst.catalog._
30-
import org.apache.spark.sql.catalyst.expressions.Expression
29+
import org.apache.spark.sql.catalyst.expressions.{And, Expression, Or}
3130
import org.apache.spark.sql.catalyst.parser._
3231
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
3332
import org.apache.spark.sql.catalyst.plans.logical._
@@ -925,9 +924,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
925924
if (ctx.VIEW != null) {
926925
operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx)
927926
}
927+
val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
928+
val conditions = ctx.dropPartitionSpec().asScala.map(visitDropPartitionSpec(_).reduce(And)).reduce(Or)
928929
AlterTableDropPartitionCommand(
929-
visitTableIdentifier(ctx.tableIdentifier),
930-
ctx.dropPartitionSpec().asScala.map(visitDropPartitionSpec),
930+
tableIdent,
931+
Filter(conditions, UnresolvedRelation(tableIdent)),
931932
ifExists = ctx.EXISTS != null,
932933
purge = ctx.PURGE != null,
933934
retainData = false)

sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala

Lines changed: 22 additions & 77 deletions
Original file line numberDiff line numberDiff line change
@@ -22,18 +22,17 @@ import java.util.Locale
2222
import scala.collection.{GenMap, GenSeq}
2323
import scala.collection.parallel.ForkJoinTaskSupport
2424
import scala.util.control.NonFatal
25-
2625
import org.apache.hadoop.conf.Configuration
2726
import org.apache.hadoop.fs._
2827
import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
29-
3028
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
3129
import org.apache.spark.sql.catalyst.TableIdentifier
32-
import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute}
30+
import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute, UnresolvedRelation}
3331
import org.apache.spark.sql.catalyst.catalog._
3432
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
3533
import org.apache.spark.sql.catalyst.expressions._
36-
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
34+
import org.apache.spark.sql.catalyst.plans.QueryPlan
35+
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
3736
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils}
3837
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
3938
import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter
@@ -523,89 +522,37 @@ case class AlterTableRenamePartitionCommand(
523522
*/
524523
case class AlterTableDropPartitionCommand(
525524
tableName: TableIdentifier,
526-
partitionsFilters: Seq[Seq[Expression]],
525+
partitionFilter: LogicalPlan,
527526
ifExists: Boolean,
528527
purge: Boolean,
529528
retainData: Boolean)
530-
extends RunnableCommand {
529+
extends RunnableCommand with PredicateHelper {
530+
531+
override def innerChildren: Seq[LogicalPlan] = partitionFilter :: Nil
531532

532533
override def run(sparkSession: SparkSession): Seq[Row] = {
533534
val catalog = sparkSession.sessionState.catalog
534-
val timeZone = Option(sparkSession.sessionState.conf.sessionLocalTimeZone)
535535
val table = catalog.getTableMetadata(tableName)
536-
val partitionColumns = table.partitionColumnNames
537-
val partitionAttributes = table.partitionSchema.toAttributes.map(a => a.name -> a).toMap
538536
DDLUtils.verifyAlterTableType(catalog, table, isView = false)
539537
DDLUtils.verifyPartitionProviderIsHive(sparkSession, table, "ALTER TABLE DROP PARTITION")
540538

541-
val resolvedSpecs = partitionsFilters.flatMap { filtersSpec =>
542-
if (hasComplexFilters(filtersSpec)) {
543-
generatePartitionSpec(filtersSpec,
544-
partitionColumns,
545-
partitionAttributes,
546-
table.identifier,
547-
catalog,
548-
sparkSession.sessionState.conf.resolver,
549-
timeZone,
550-
ifExists)
551-
} else {
552-
val partitionSpec = filtersSpec.map {
553-
case EqualTo(key: Attribute, Literal(value, StringType)) =>
554-
key.name -> value.toString
555-
}.toMap
556-
PartitioningUtils.normalizePartitionSpec(
557-
partitionSpec,
558-
partitionColumns,
559-
table.identifier.quotedString,
560-
sparkSession.sessionState.conf.resolver) :: Nil
561-
}
539+
val resolvedSpec = sparkSession.sessionState.analyzer.executeAndCheck(partitionFilter) match {
540+
case Filter(conditions, _) =>
541+
val filters = splitDisjunctivePredicates(conditions).flatMap { filter =>
542+
splitConjunctivePredicates(filter)
543+
}
544+
// Resolve TablePartitionSpec based on the resolved expressions?
545+
...
562546
}
563547

564548
catalog.dropPartitions(
565-
table.identifier, resolvedSpecs, ignoreIfNotExists = ifExists, purge = purge,
549+
table.identifier, resolvedSpec, ignoreIfNotExists = ifExists, purge = purge,
566550
retainData = retainData)
567551

568552
CommandUtils.updateTableStats(sparkSession, table)
569553

570554
Seq.empty[Row]
571555
}
572-
573-
def hasComplexFilters(partitionFilterSpec: Seq[Expression]): Boolean = {
574-
partitionFilterSpec.exists(!_.isInstanceOf[EqualTo])
575-
}
576-
577-
def generatePartitionSpec(
578-
partitionFilterSpec: Seq[Expression],
579-
partitionColumns: Seq[String],
580-
partitionAttributes: Map[String, Attribute],
581-
tableIdentifier: TableIdentifier,
582-
catalog: SessionCatalog,
583-
resolver: Resolver,
584-
timeZone: Option[String],
585-
ifExists: Boolean): Seq[TablePartitionSpec] = {
586-
val filters = partitionFilterSpec.map { pFilter =>
587-
pFilter.transform {
588-
// Resolve the partition attributes
589-
case partitionCol: Attribute =>
590-
val normalizedPartition = PartitioningUtils.normalizePartitionColumn(
591-
partitionCol.name,
592-
partitionColumns,
593-
tableIdentifier.quotedString,
594-
resolver)
595-
partitionAttributes(normalizedPartition)
596-
}.transform {
597-
// Cast the partition value to the data type of the corresponding partition attribute
598-
case cmp @ BinaryComparison(partitionAttr, value)
599-
if !partitionAttr.dataType.sameType(value.dataType) =>
600-
cmp.withNewChildren(Seq(partitionAttr, Cast(value, partitionAttr.dataType, timeZone)))
601-
}
602-
}
603-
val partitions = catalog.listPartitionsByFilter(tableIdentifier, filters)
604-
if (partitions.isEmpty && !ifExists) {
605-
throw new AnalysisException(s"There is no partition for ${filters.reduceLeft(And).sql}")
606-
}
607-
partitions.map(_.spec)
608-
}
609556
}
610557

611558

@@ -617,17 +564,15 @@ object AlterTableDropPartitionCommand {
617564
ifExists: Boolean,
618565
purge: Boolean,
619566
retainData: Boolean): AlterTableDropPartitionCommand = {
620-
AlterTableDropPartitionCommand(tableName,
621-
specs.map(tablePartitionToPartitionFilters),
622-
ifExists,
623-
purge,
624-
retainData)
567+
val conditions = specs.map(tablePartitionToPartitionFilter)
568+
val filter = Filter(conditions.reduce(Or), UnresolvedRelation(tableName))
569+
AlterTableDropPartitionCommand(tableName, filter, ifExists, purge, retainData)
625570
}
626571

627-
def tablePartitionToPartitionFilters(spec: TablePartitionSpec): Seq[Expression] = {
628-
spec.map {
629-
case (key, value) => EqualTo(AttributeReference(key, StringType)(), Literal(value))
630-
}.toSeq
572+
private def tablePartitionToPartitionFilter(spec: TablePartitionSpec): Expression = {
573+
spec.map { case (key, value) =>
574+
EqualTo(AttributeReference(key, StringType)(), Literal(value))
575+
}.reduce(And)
631576
}
632577
}
633578

0 commit comments

Comments
 (0)