This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 3a92fb338ca [HUDI-8495] Add drop_partition procedure (#12222)
3a92fb338ca is described below

commit 3a92fb338cafcad59bb4c45bd71a977d8ae3a00c
Author: fhan <[email protected]>
AuthorDate: Tue Nov 12 09:12:04 2024 +0800

    [HUDI-8495] Add drop_partition procedure (#12222)
---
 .../hudi/command/procedures/BaseProcedure.scala    |  10 +
 .../procedures/DropPartitionProcedure.scala        |  82 +++++++
 .../hudi/command/procedures/HoodieProcedures.scala |   1 +
 .../procedure/TestDropPartitionProcedure.scala     | 237 +++++++++++++++++++++
 4 files changed, 330 insertions(+)

diff --git 
a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala
 
b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala
index 777d1937c98..1b5494814df 100644
--- 
a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala
+++ 
b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala
@@ -109,4 +109,14 @@ abstract class BaseProcedure extends Procedure {
       )
   }
 
+  protected def getDbAndTableName(tableName: String): (String, String) = {
+    val names = tableName.split("\\.")
+    if (names.length == 1) {
+      ("default", names(0))
+    } else if (names.length == 2) {
+      (names(0), names(1))
+    } else {
+      throw new HoodieException(s"Table name: $tableName is not valid")
+    }
+  }
 }
diff --git 
a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DropPartitionProcedure.scala
 
b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DropPartitionProcedure.scala
new file mode 100644
index 00000000000..d6995be261f
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DropPartitionProcedure.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi.command.procedures
+
+import org.apache.hudi.HoodieSparkSqlWriter
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
+import org.apache.spark.sql.{Row, SaveMode}
+import org.apache.spark.sql.hudi.ProvidesHoodieConfig
+import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, 
StructType}
+
+import java.util.function.Supplier
+
+class DropPartitionProcedure extends BaseProcedure
+  with ProcedureBuilder
+  with Logging
+  with ProvidesHoodieConfig {
+  override def build: Procedure = new DropPartitionProcedure
+
+  val PARAMETERS: Array[ProcedureParameter] = Array[ProcedureParameter](
+    ProcedureParameter.required(0, "table", DataTypes.StringType),
+    ProcedureParameter.required(1, "partition", DataTypes.StringType)
+  )
+
+  override def parameters: Array[ProcedureParameter] = PARAMETERS
+
+  override def outputType: StructType = new StructType(Array[StructField](
+    StructField("result", DataTypes.StringType, nullable = true, 
Metadata.empty)
+  ))
+
+  override def call(args: ProcedureArgs): Seq[Row] = {
+    super.checkArgs(PARAMETERS, args)
+
+    val tableName = getArgValueOrDefault(args, PARAMETERS(0))
+    val partitions = getArgValueOrDefault(args, PARAMETERS(1))
+    val tableNameStr = tableName.get.asInstanceOf[String]
+    val partitionsStr = partitions.get.asInstanceOf[String]
+
+    val (db, table) = getDbAndTableName(tableNameStr)
+
+    val hoodieCatalogTable = HoodieCatalogTable(sparkSession, 
TableIdentifier(table, Some(db)))
+
+    val parameters = buildHoodieDropPartitionsConfig(sparkSession, 
hoodieCatalogTable, partitionsStr)
+    val (success, _, _, _, _, _) = HoodieSparkSqlWriter.write(
+      sparkSession.sqlContext,
+      SaveMode.Append,
+      parameters,
+      sparkSession.emptyDataFrame)
+
+    if (!success) {
+      throw new RuntimeException(s"Failed to drop partition $partitionsStr for 
table $tableNameStr")
+    }
+
+    sparkSession.catalog.refreshTable(tableNameStr)
+    logInfo(s"Finish execute alter table drop partition procedure for 
$tableNameStr")
+    Seq(Row("Success"))
+  }
+}
+
+object DropPartitionProcedure {
+  val NAME: String = "drop_partition"
+
+  def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
+    override def get() = new DropPartitionProcedure
+  }
+}
diff --git 
a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala
 
b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala
index 87c6971a791..501bdbc2da0 100644
--- 
a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala
+++ 
b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala
@@ -94,6 +94,7 @@ object HoodieProcedures {
       ,(HelpProcedure.NAME, HelpProcedure.builder)
       ,(ArchiveCommitsProcedure.NAME, ArchiveCommitsProcedure.builder)
       ,(RunTTLProcedure.NAME, RunTTLProcedure.builder)
+      ,(DropPartitionProcedure.NAME, DropPartitionProcedure.builder)
     )
   }
 }
diff --git 
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestDropPartitionProcedure.scala
 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestDropPartitionProcedure.scala
new file mode 100644
index 00000000000..bc8c8ebeabb
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestDropPartitionProcedure.scala
@@ -0,0 +1,237 @@
+/*
+ * 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.hudi.procedure
+
+import org.apache.hudi.common.model.HoodieReplaceCommitMetadata
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.hadoop.fs.HadoopFSUtils
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+
+import scala.collection.JavaConverters._
+
+class TestDropPartitionProcedure extends HoodieSparkProcedureTestBase {
+
+  test("Case1: Test Call drop_partition Procedure For Multiple Partitions: '*' 
stands for all partitions in leaf partition") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      val tablePath = tmp.getCanonicalPath + "/" + tableName
+      // create table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  year string,
+           |  month string,
+           |  day string
+           |) using hudi
+           |tblproperties (
+           | primaryKey = 'id',
+           | preCombineField = 'ts'
+           |)
+           |partitioned by(year, month, day)
+           |location '$tablePath'
+           |
+     """.stripMargin)
+
+      insertData(tableName)
+
+      spark.sql(s"""call drop_partition(table => '$tableName', partition => 
'year=2019/month=08/*')""")
+
+      val metaClient = getTableMetaClient(tablePath)
+      val replaceCommitInstant = metaClient.getActiveTimeline.getWriteTimeline
+        .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst()
+        .get()
+
+      val partitions = HoodieReplaceCommitMetadata
+        
.fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(),
 classOf[HoodieReplaceCommitMetadata])
+        .getPartitionToReplaceFileIds
+        .keySet()
+
+      assertEquals(3, partitions.size())
+      assertTrue(partitions.asScala.forall(_.startsWith("year=2019/month=08")))
+
+      // clean
+      spark.sql(s"""call run_clean(table => '$tableName', clean_policy => 
'KEEP_LATEST_FILE_VERSIONS', file_versions_retained => 1)""")
+      val result = spark.sql(s"""select * from $tableName""").collect()
+      assertEquals(1, result.length)
+    }
+  }
+
+  test("Case2: Test Call drop_partition Procedure For Multiple Partitions: '*' 
stands for all partitions in middle partition") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      val tablePath = tmp.getCanonicalPath + "/" + tableName
+      // create table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  year string,
+           |  month string,
+           |  day string
+           |) using hudi
+           |tblproperties (
+           | primaryKey = 'id',
+           | preCombineField = 'ts'
+           |)
+           |partitioned by(year, month, day)
+           |location '$tablePath'
+           |
+     """.stripMargin)
+
+      insertData(tableName)
+
+      spark.sql(s"""call drop_partition(table => '$tableName', partition => 
'year=2019/*')""")
+
+      val metaClient = getTableMetaClient(tablePath)
+      val replaceCommitInstant = metaClient.getActiveTimeline.getWriteTimeline
+        .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst()
+        .get()
+
+      val partitions = HoodieReplaceCommitMetadata
+        
.fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(),
 classOf[HoodieReplaceCommitMetadata])
+        .getPartitionToReplaceFileIds
+        .keySet()
+
+      assertEquals(4, partitions.size())
+
+      // clean
+      spark.sql(s"""call run_clean(table => '$tableName', clean_policy => 
'KEEP_LATEST_FILE_VERSIONS', file_versions_retained => 1)""")
+      val result = spark.sql(s"""select * from $tableName""").collect()
+      assertEquals(0, result.length)
+    }
+  }
+
+  test("Case3: Test Call drop_partition Procedure For Multiple Partitions: 
provide partition list") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      val tablePath = tmp.getCanonicalPath + "/" + tableName
+      // create table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  year string,
+           |  month string,
+           |  day string
+           |) using hudi
+           |tblproperties (
+           | primaryKey = 'id',
+           | preCombineField = 'ts'
+           |)
+           |partitioned by(year, month, day)
+           |location '$tablePath'
+           |
+     """.stripMargin)
+
+      insertData(tableName)
+
+      spark.sql(s"""call drop_partition(table => '$tableName', partition => 
'year=2019/month=08/day=31,year=2019/month=08/day=30')""")
+
+      val metaClient = getTableMetaClient(tablePath)
+      val replaceCommitInstant = metaClient.getActiveTimeline.getWriteTimeline
+        .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst()
+        .get()
+
+      val partitions = HoodieReplaceCommitMetadata
+        
.fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(),
 classOf[HoodieReplaceCommitMetadata])
+        .getPartitionToReplaceFileIds
+        .keySet()
+
+      assertEquals(2, partitions.size())
+      assertTrue(partitions.asScala.forall(_.startsWith("year=2019/month=08")))
+
+      // clean
+      spark.sql(s"""call run_clean(table => '$tableName', clean_policy => 
'KEEP_LATEST_FILE_VERSIONS', file_versions_retained => 1)""")
+      val result = spark.sql(s"""select * from $tableName""").collect()
+      assertEquals(2, result.length)
+    }
+  }
+
+  test("Case4: Test Call drop_partition Procedure For Single Partition") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      val tablePath = tmp.getCanonicalPath + "/" + tableName
+      // create table
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long,
+           |  year string,
+           |  month string,
+           |  day string
+           |) using hudi
+           |tblproperties (
+           | primaryKey = 'id',
+           | preCombineField = 'ts'
+           |)
+           |partitioned by(year, month, day)
+           |location '$tablePath'
+           |
+     """.stripMargin)
+
+      insertData(tableName)
+
+      spark.sql(s"""call drop_partition(table => '$tableName', partition => 
'year=2019/month=08/day=31')""")
+
+      val metaClient = getTableMetaClient(tablePath)
+      val replaceCommitInstant = metaClient.getActiveTimeline.getWriteTimeline
+        .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst()
+        .get()
+
+      val partitions = HoodieReplaceCommitMetadata
+        
.fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(),
 classOf[HoodieReplaceCommitMetadata])
+        .getPartitionToReplaceFileIds
+        .keySet()
+
+      assertEquals(1, partitions.size())
+      
assertTrue(partitions.asScala.forall(_.equals("year=2019/month=08/day=31")))
+
+      // clean
+      spark.sql(s"""call run_clean(table => '$tableName', clean_policy => 
'KEEP_LATEST_FILE_VERSIONS', file_versions_retained => 1)""")
+      val result = spark.sql(s"""select * from $tableName""").collect()
+      assertEquals(3, result.length)
+    }
+  }
+
+  private def insertData(tableName: String): Unit = {
+    spark.sql(s"""insert into $tableName values (1, 'n1', 1, 1, '2019', '08', 
'31')""")
+    spark.sql(s"""insert into $tableName values (2, 'n2', 2, 2, '2019', '08', 
'30')""")
+    spark.sql(s"""insert into $tableName values (3, 'n3', 3, 3, '2019', '08', 
'29')""")
+    spark.sql(s"""insert into $tableName values (4, 'n4', 4, 4, '2019', '07', 
'31')""")
+  }
+
+  private def getTableMetaClient(tablePath: String): HoodieTableMetaClient = {
+    HoodieTableMetaClient.builder()
+      .setBasePath(tablePath)
+      
.setConf(HadoopFSUtils.getStorageConf(spark.sparkContext.hadoopConfiguration))
+      .build()
+  }
+}

Reply via email to