diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 6fe995f650d5..53d92989a778 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -250,7 +250,7 @@ partitionSpecLocation ; partitionSpec - : PARTITION '(' partitionVal (',' partitionVal)* ')' + : PARTITION '(' (partitionVal | expression) (',' (partitionVal | expression))* ')' ; partitionVal 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 7651d11ee65a..0772dcf471b4 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 @@ -282,6 +282,26 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging parts.toMap } + /** + * Create a partition filter specification. + */ + def visitPartitionFilterSpec(ctx: PartitionSpecContext): Seq[Expression] = withOrigin(ctx) { + val parts = ctx.expression.asScala.map { pVal => + expression(pVal) match { + case EqualNullSafe(_, _) => + throw new ParseException("'<=>' operator is not allowed in partition specification.", ctx) + case cmp @ BinaryComparison(UnresolvedAttribute(name :: Nil), constant: Literal) => + cmp + case bc @ BinaryComparison(constant: Literal, _) => + throw new ParseException(s"Literal $constant is supported only on the rigth-side.", ctx) + case _ => + throw new ParseException( + s"Invalid partition filter specification (${pVal.getText}).", ctx) + } + } + parts + } + /** * Create a partition specification map without optional values. */ @@ -293,6 +313,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create a partition specification map without optional values + * and a partition filter specification. + */ + protected def visitPartition( + ctx: PartitionSpecContext): (Map[String, String], Seq[Expression]) = { + (visitNonOptionalPartitionSpec(ctx), visitPartitionFilterSpec(ctx)) + } + /** * Convert a constant of any type into a string. This is typically used in DDL commands, and its * main purpose is to prevent slight differences due to back to back conversions i.e.: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 29b584b55972..910a1a76af80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -916,7 +916,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } AlterTableDropPartitionCommand( visitTableIdentifier(ctx.tableIdentifier), - ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec), + ctx.partitionSpec.asScala.map(visitPartition), ifExists = ctx.EXISTS != null, purge = ctx.PURGE != null, retainData = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 568567aa8ea8..0d1fa4e97f62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -29,10 +29,10 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver} +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver, UnresolvedAttribute} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BinaryComparison, Cast, Expression, Literal, PredicateHelper} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat @@ -500,7 +500,8 @@ case class AlterTableRenamePartitionCommand( } /** - * Drop Partition in ALTER TABLE: to drop a particular partition for a table. + * Drop Partition in ALTER TABLE: to drop a particular partition + * or a set of partitions according to given expressions for a table. * * This removes the data and metadata for this partition. * The data is actually moved to the .Trash/Current directory if Trash is configured, @@ -510,33 +511,43 @@ case class AlterTableRenamePartitionCommand( * * The syntax of this command is: * {{{ - * ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; + * ALTER TABLE table DROP [IF EXISTS] PARTITION (spec1, expr1) + * [, PARTITION (spec2, expr2), ...] [PURGE]; * }}} */ case class AlterTableDropPartitionCommand( tableName: TableIdentifier, - specs: Seq[TablePartitionSpec], + partitions: Seq[(TablePartitionSpec, Seq[Expression])], ifExists: Boolean, purge: Boolean, retainData: Boolean) - extends RunnableCommand { + extends RunnableCommand with PredicateHelper { override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) + val resolver = sparkSession.sessionState.conf.resolver DDLUtils.verifyAlterTableType(catalog, table, isView = false) DDLUtils.verifyPartitionProviderIsHive(sparkSession, table, "ALTER TABLE DROP PARTITION") - val normalizedSpecs = specs.map { spec => - PartitioningUtils.normalizePartitionSpec( - spec, - table.partitionColumnNames, - table.identifier.quotedString, - sparkSession.sessionState.conf.resolver) + val toDrop = partitions.flatMap { partition => + if (partition._1.isEmpty && !partition._2.isEmpty) { + // There are only expressions in this drop condition. + extractFromPartitionFilter(partition._2, catalog, table, resolver) + } else if (!partition._1.isEmpty && partition._2.isEmpty) { + // There are only partitionSpecs in this drop condition. + extractFromPartitionSpec(partition._1, table, resolver) + } else if (!partition._1.isEmpty && !partition._2.isEmpty) { + // This drop condition has both partitionSpecs and expressions. + extractFromPartitionFilter(partition._2, catalog, table, resolver).intersect( + extractFromPartitionSpec(partition._1, table, resolver)) + } else { + Seq.empty[TablePartitionSpec] + } } catalog.dropPartitions( - table.identifier, normalizedSpecs, ignoreIfNotExists = ifExists, purge = purge, + table.identifier, toDrop, ignoreIfNotExists = ifExists, purge = purge, retainData = retainData) CommandUtils.updateTableStats(sparkSession, table) @@ -544,6 +555,42 @@ case class AlterTableDropPartitionCommand( Seq.empty[Row] } + private def extractFromPartitionSpec( + specs: TablePartitionSpec, + table: CatalogTable, + resolver: Resolver): Seq[Map[String, String]] = { + Seq(PartitioningUtils.normalizePartitionSpec( + specs, + table.partitionColumnNames, + table.identifier.quotedString, + resolver)) + } + + private def extractFromPartitionFilter( + filters: Seq[Expression], + catalog: SessionCatalog, + table: CatalogTable, + resolver: Resolver): Seq[TablePartitionSpec] = { + val expressions = filters.map { expr => + val (attrName, constant) = expr match { + case BinaryComparison(UnresolvedAttribute(name :: Nil), constant: Literal) => + (name, constant) + } + if (!table.partitionColumnNames.exists(resolver(_, attrName))) { + throw new AnalysisException(s"${attrName} is not a valid partition column " + + s"in table ${table.identifier.quotedString}.") + } + val dataType = table.partitionSchema.apply(attrName).dataType + expr.withNewChildren(Seq(AttributeReference(attrName, dataType)(), + Cast(constant, dataType))) + }.reduce(And) + val parts = catalog.listPartitionsByFilter( + table.identifier, Seq(expressions)).map(_.spec) + if (parts.isEmpty && !ifExists) { + throw new AnalysisException(s"There is no partition for ${expressions.sql}") + } + parts + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 675bee85bf61..39519276b8d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -25,7 +25,7 @@ import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.util.SchemaUtils @@ -128,7 +128,7 @@ case class InsertIntoHadoopFsRelationCommand( val deletedPartitions = initialMatchingPartitions.toSet -- updatedPartitions if (deletedPartitions.nonEmpty) { AlterTableDropPartitionCommand( - catalogTable.get.identifier, deletedPartitions.toSeq, + catalogTable.get.identifier, deletedPartitions.map(x => (x, Seq.empty)).toSeq, ifExists = true, purge = false, retainData = true /* already deleted */).run(sparkSession) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index eb7c33590b60..3b21cb1199db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -826,8 +826,8 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { val expected1_table = AlterTableDropPartitionCommand( tableIdent, Seq( - Map("dt" -> "2008-08-08", "country" -> "us"), - Map("dt" -> "2009-09-09", "country" -> "uk")), + (Map("dt" -> "2008-08-08", "country" -> "us"), Seq.empty), + (Map("dt" -> "2009-09-09", "country" -> "uk"), Seq.empty)), ifExists = true, purge = false, retainData = false) 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 d3465a641a1a..b3da4625f6c1 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 @@ -32,6 +32,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.HiveUtils.{CONVERT_METASTORE_ORC, CONVERT_METASTORE_PARQUET} @@ -495,6 +496,164 @@ class HiveDDLSuite } } + def testDropPartition(dataType: DataType, value1: Any, value2: Any): Unit = { + withTable("tbl_x") { + sql(s"CREATE TABLE tbl_x (a INT) PARTITIONED BY (p ${dataType.sql})") + sql(s"ALTER TABLE tbl_x ADD PARTITION (p = $value1)") + sql(s"ALTER TABLE tbl_x ADD PARTITION (p = $value2)") + sql(s"ALTER TABLE tbl_x DROP PARTITION (p >= $value2)") + checkAnswer(sql("SHOW PARTITIONS tbl_x"), + Row(s"p=$value1") :: Nil) + sql(s"ALTER TABLE tbl_x DROP PARTITION (p = $value1)") + checkAnswer(sql("SHOW PARTITIONS tbl_x"), Nil) + } + } + + test("SPARK-17732: Drop partitions by filter") { + withTable("sales") { + sql("CREATE TABLE sales (id INT) PARTITIONED BY (country STRING, quarter STRING)") + + for (country <- Seq("AU", "US", "CA", "KR")) { + for (quarter <- 1 to 5) { + sql(s"ALTER TABLE sales ADD PARTITION (country = '$country', quarter = '$quarter')") + } + } + + sql("ALTER TABLE sales DROP PARTITION (country < 'KR', quarter > '2')") + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=AU/quarter=1") :: + Row("country=AU/quarter=2") :: + Row("country=CA/quarter=1") :: + Row("country=CA/quarter=2") :: + Row("country=KR/quarter=1") :: + Row("country=KR/quarter=2") :: + Row("country=KR/quarter=3") :: + Row("country=KR/quarter=4") :: + Row("country=KR/quarter=5") :: + Row("country=US/quarter=1") :: + Row("country=US/quarter=2") :: + Row("country=US/quarter=3") :: + Row("country=US/quarter=4") :: + Row("country=US/quarter=5") :: Nil) + + sql("ALTER TABLE sales DROP PARTITION (country < 'CA'), PARTITION (quarter = '5')") + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=CA/quarter=1") :: + Row("country=CA/quarter=2") :: + Row("country=KR/quarter=1") :: + Row("country=KR/quarter=2") :: + Row("country=KR/quarter=3") :: + Row("country=KR/quarter=4") :: + Row("country=US/quarter=1") :: + Row("country=US/quarter=2") :: + Row("country=US/quarter=3") :: + Row("country=US/quarter=4") :: Nil) + + sql("ALTER TABLE sales DROP PARTITION (country < 'KR'), PARTITION (quarter <= '1')") + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=KR/quarter=2") :: + Row("country=KR/quarter=3") :: + Row("country=KR/quarter=4") :: + Row("country=US/quarter=2") :: + Row("country=US/quarter=3") :: + Row("country=US/quarter=4") :: Nil) + + sql("ALTER TABLE sales DROP PARTITION (country = 'KR', quarter = '4')") + sql("ALTER TABLE sales DROP PARTITION (country = 'US', quarter = '3')") + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=KR/quarter=2") :: + Row("country=KR/quarter=3") :: + Row("country=US/quarter=2") :: + Row("country=US/quarter=4") :: Nil) + + sql("ALTER TABLE sales DROP PARTITION (quarter <= '2'), PARTITION (quarter >= '4')") + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=KR/quarter=3") :: Nil) + + // According to the declarative partition spec definitions, this drops the union of target + // partitions without exceptions. Hive raises exceptions because it handles them sequentially. + sql("ALTER TABLE sales DROP PARTITION (quarter <= '4'), PARTITION (quarter <= '3')") + checkAnswer(sql("SHOW PARTITIONS sales"), Nil) + } + + withTable("tbl_x") { + sql(s"CREATE TABLE tbl_x (a INT) PARTITIONED BY (p STRING)") + sql(s"ALTER TABLE tbl_x ADD PARTITION (p = 'false')") + sql(s"ALTER TABLE tbl_x ADD PARTITION (p = 'true')") + sql(s"ALTER TABLE tbl_x DROP PARTITION (p >= 'true')") + checkAnswer(sql("SHOW PARTITIONS tbl_x"), + Row(s"p=false") :: Nil) + sql(s"ALTER TABLE tbl_x DROP PARTITION (p = 'false')") + checkAnswer(sql("SHOW PARTITIONS tbl_x"), Nil) + } + testDropPartition(IntegerType, 1, 2) + testDropPartition(BooleanType, false, true) + testDropPartition(LongType, 1L, 2L) + testDropPartition(ShortType, 1.toShort, 2.toShort) + testDropPartition(ByteType, 1.toByte, 2.toByte) + testDropPartition(FloatType, 1.0F, 2.0F) + testDropPartition(DoubleType, 1.0, 2.0) + testDropPartition(DecimalType(2, 1), Decimal(1.5), Decimal(2.5)) + } + + test("SPARK-14922, SPARK-17732: Error handling for drop partitions by filter") { + withTable("sales") { + sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)") + + val m = intercept[AnalysisException] { + sql("ALTER TABLE sales DROP PARTITION (unknown = 'KR')") + }.getMessage + assert(m.contains("unknown is not a valid partition column in table")) + + val m2 = intercept[AnalysisException] { + sql("ALTER TABLE sales DROP PARTITION (unknown < 'KR')") + }.getMessage + assert(m2.contains("unknown is not a valid partition column in table")) + + val m3 = intercept[AnalysisException] { + sql("ALTER TABLE sales DROP PARTITION (unknown <=> 'KR')") + }.getMessage + assert(m3.contains("'<=>' operator is not allowed in partition specification")) + + val m4 = intercept[ParseException] { + sql("ALTER TABLE sales DROP PARTITION (unknown <=> upper('KR'))") + }.getMessage + assert(m4.contains("'<=>' operator is not allowed in partition specification")) + + val m5 = intercept[ParseException] { + sql("ALTER TABLE sales DROP PARTITION (country < 'KR', quarter)") + }.getMessage + assert(m5.contains("Found an empty partition key")) + + sql(s"ALTER TABLE sales ADD PARTITION (country = 'KR', quarter = '3')") + val m6 = intercept[AnalysisException] { + sql("ALTER TABLE sales DROP PARTITION (quarter <= '4'), PARTITION (quarter <= '2')") + }.getMessage + // The query is not executed because `PARTITION (quarter <= '2')` is invalid. + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=KR/quarter=3") :: Nil) + assert(m6.contains("There is no partition for (`quarter` <= CAST('2' AS STRING))")) + + val m7 = intercept[ParseException] { + sql("ALTER TABLE sales DROP PARTITION ( '4' > quarter)") + }.getMessage + checkAnswer(sql("SHOW PARTITIONS sales"), + Row("country=KR/quarter=3") :: Nil) + assert(m7.contains("Literal 4 is supported only on the rigth-side")) + } + } + + test("SPARK-17732: Partition filter is not allowed in ADD PARTITION") { + withTable("sales") { + sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)") + + val m = intercept[AnalysisException] { + sql("ALTER TABLE sales ADD PARTITION (country = 'US', quarter < '1')") + }.getMessage + assert(m.contains("Partition spec is invalid")) + } + } + test("drop views") { withTable("tab1") { val tabName = "tab1"