@@ -22,18 +22,17 @@ import java.util.Locale
2222import scala .collection .{GenMap , GenSeq }
2323import scala .collection .parallel .ForkJoinTaskSupport
2424import scala .util .control .NonFatal
25-
2625import org .apache .hadoop .conf .Configuration
2726import org .apache .hadoop .fs ._
2827import org .apache .hadoop .mapred .{FileInputFormat , JobConf }
29-
3028import org .apache .spark .sql .{AnalysisException , Row , SparkSession }
3129import 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 }
3331import org .apache .spark .sql .catalyst .catalog ._
3432import org .apache .spark .sql .catalyst .catalog .CatalogTypes .TablePartitionSpec
3533import 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 }
3736import org .apache .spark .sql .execution .datasources .{HadoopFsRelation , LogicalRelation , PartitioningUtils }
3837import org .apache .spark .sql .execution .datasources .orc .OrcFileFormat
3938import org .apache .spark .sql .execution .datasources .parquet .ParquetSchemaConverter
@@ -523,89 +522,37 @@ case class AlterTableRenamePartitionCommand(
523522 */
524523case 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