[HUDI-3826] Make truncate partition use delete_partition operation (#5272)
Make truncate partition and drop partition behave as drop partition with purge, which delete all records via Hudi DELETE_PARTITION; partition removed from metastore
This commit is contained in:
@@ -539,7 +539,7 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic
|
||||
// Rewrite TruncateTableCommand to TruncateHoodieTableCommand
|
||||
case TruncateTableCommand(tableName, partitionSpec)
|
||||
if sparkAdapter.isHoodieTable(tableName, sparkSession) =>
|
||||
new TruncateHoodieTableCommand(tableName, partitionSpec)
|
||||
TruncateHoodieTableCommand(tableName, partitionSpec)
|
||||
case _ => plan
|
||||
}
|
||||
}
|
||||
|
||||
@@ -52,7 +52,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|
||||
checkAnswer(s"show partitions $tableName")(Seq.empty: _*)
|
||||
}
|
||||
|
||||
test("Purge drop non-partitioned table") {
|
||||
test("Lazy Clean drop non-partitioned table") {
|
||||
val tableName = generateTableName
|
||||
// create table
|
||||
spark.sql(
|
||||
@@ -66,13 +66,14 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|
||||
| using hudi
|
||||
| tblproperties (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts'
|
||||
| preCombineField = 'ts',
|
||||
| hoodie.cleaner.commits.retained= '1'
|
||||
| )
|
||||
|""".stripMargin)
|
||||
// insert data
|
||||
spark.sql(s"""insert into $tableName values (1, "z3", "v1", "2021-10-01"), (2, "l4", "v1", "2021-10-02")""")
|
||||
|
||||
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01') purge")(
|
||||
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01')")(
|
||||
s"$tableName is a non-partitioned table that is not allowed to drop partition")
|
||||
|
||||
// show partitions
|
||||
@@ -131,14 +132,13 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|
||||
}
|
||||
|
||||
Seq(false, true).foreach { urlencode =>
|
||||
test(s"Purge drop single-partition table' partitions, urlencode: $urlencode") {
|
||||
test(s"Lazy Clean drop single-partition table' partitions, urlencode: $urlencode") {
|
||||
withTempDir { tmp =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${tmp.getCanonicalPath}/$tableName"
|
||||
|
||||
import spark.implicits._
|
||||
val df = Seq((1, "z3", "v1", "2021/10/01"), (2, "l4", "v1", "2021/10/02"))
|
||||
.toDF("id", "name", "ts", "dt")
|
||||
val df = Seq((1, "z3", "v1", "2021/10/01")).toDF("id", "name", "ts", "dt")
|
||||
|
||||
df.write.format("hudi")
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
@@ -158,17 +158,24 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|
||||
s"""
|
||||
|create table $tableName using hudi
|
||||
|location '$tablePath'
|
||||
| tblproperties (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts',
|
||||
| hoodie.cleaner.commits.retained= '1'
|
||||
| )
|
||||
|""".stripMargin)
|
||||
|
||||
// drop 2021-10-01 partition
|
||||
spark.sql(s"alter table $tableName drop partition (dt='2021/10/01') purge")
|
||||
spark.sql(s"alter table $tableName drop partition (dt='2021/10/01')")
|
||||
|
||||
spark.sql(s"""insert into $tableName values (2, "l4", "v1", "2021/10/02")""")
|
||||
|
||||
val partitionPath = if (urlencode) {
|
||||
PartitionPathEncodeUtils.escapePathName("2021/10/01")
|
||||
} else {
|
||||
"2021/10/01"
|
||||
}
|
||||
checkAnswer(s"select dt from $tableName")(Seq(s"2021/10/02"))
|
||||
checkAnswer(s"select dt from $tableName")(Seq("2021/10/02"))
|
||||
assertResult(false)(existsPath(s"${tmp.getCanonicalPath}/$tableName/$partitionPath"))
|
||||
|
||||
// show partitions
|
||||
@@ -267,14 +274,13 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|
||||
}
|
||||
|
||||
Seq(false, true).foreach { hiveStyle =>
|
||||
test(s"Purge drop multi-level partitioned table's partitions, isHiveStylePartitioning: $hiveStyle") {
|
||||
test(s"Lazy Clean drop multi-level partitioned table's partitions, isHiveStylePartitioning: $hiveStyle") {
|
||||
withTempDir { tmp =>
|
||||
val tableName = generateTableName
|
||||
val tablePath = s"${tmp.getCanonicalPath}/$tableName"
|
||||
|
||||
import spark.implicits._
|
||||
val df = Seq((1, "z3", "v1", "2021", "10", "01"), (2, "l4", "v1", "2021", "10", "02"))
|
||||
.toDF("id", "name", "ts", "year", "month", "day")
|
||||
val df = Seq((1, "z3", "v1", "2021", "10", "01")).toDF("id", "name", "ts", "year", "month", "day")
|
||||
|
||||
df.write.format("hudi")
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
@@ -294,14 +300,23 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase {
|
||||
s"""
|
||||
|create table $tableName using hudi
|
||||
|location '$tablePath'
|
||||
| tblproperties (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts',
|
||||
| hoodie.cleaner.commits.retained= '1'
|
||||
| )
|
||||
|""".stripMargin)
|
||||
|
||||
// drop 2021-10-01 partition
|
||||
spark.sql(s"alter table $tableName drop partition (year='2021', month='10', day='01') purge")
|
||||
spark.sql(s"alter table $tableName drop partition (year='2021', month='10', day='01')")
|
||||
|
||||
// insert data
|
||||
spark.sql(s"""insert into $tableName values (2, "l4", "v1", "2021", "10", "02")""")
|
||||
|
||||
checkAnswer(s"select id, name, ts, year, month, day from $tableName")(
|
||||
Seq(2, "l4", "v1", "2021", "10", "02")
|
||||
)
|
||||
|
||||
assertResult(false)(existsPath(
|
||||
s"${tmp.getCanonicalPath}/$tableName/year=2021/month=10/day=01"))
|
||||
|
||||
|
||||
Reference in New Issue
Block a user