[HUDI-2307] When using delete_partition with ds should not rely on the primary key (#3469)
- Co-authored-by: Sivabalan Narayanan <n.siva.b@gmail.com>
This commit is contained in:
@@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.config.HoodieConfig
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload, HoodieTableType, WriteOperationType}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
@@ -679,61 +679,68 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
}
|
||||
}
|
||||
|
||||
test("test delete partitions") {
|
||||
initSparkContext("test_delete_partitions")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path_delete_partitions")
|
||||
try {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl_delete_partitions"
|
||||
val fooTableModifier = getCommonParams(path, hoodieFooTableName, HoodieTableType.COPY_ON_WRITE.name())
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(10)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1)
|
||||
List(true, false)
|
||||
.foreach(usePartitionsToDeleteConfig => {
|
||||
test("test delete partitions for " + usePartitionsToDeleteConfig) {
|
||||
initSparkContext("test_delete_partitions_" + usePartitionsToDeleteConfig)
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path_delete_partitions")
|
||||
try {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl_delete_partitions"
|
||||
val fooTableModifier = getCommonParams(path, hoodieFooTableName, HoodieTableType.COPY_ON_WRITE.name())
|
||||
var fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(10)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1)
|
||||
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(10, snapshotDF1.count())
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf1 = dropMetaFields(snapshotDF1)
|
||||
assert(df1.except(trimmedDf1).count() == 0)
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(10, snapshotDF1.count())
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf1 = dropMetaFields(snapshotDF1)
|
||||
assert(df1.except(trimmedDf1).count() == 0)
|
||||
|
||||
// issue updates so that log files are created for MOR table
|
||||
var updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5))
|
||||
var updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
|
||||
// write updates to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf)
|
||||
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(10, snapshotDF2.count())
|
||||
// issue updates so that log files are created for MOR table
|
||||
var updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5))
|
||||
var updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
|
||||
// write updates to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf)
|
||||
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(10, snapshotDF2.count())
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf2 = dropMetaFields(snapshotDF2)
|
||||
// ensure 2nd batch of updates matches.
|
||||
assert(updatesDf.intersect(trimmedDf2).except(updatesDf).count() == 0)
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf2 = dropMetaFields(snapshotDF2)
|
||||
// ensure 2nd batch of updates matches.
|
||||
assert(updatesDf.intersect(trimmedDf2).except(updatesDf).count() == 0)
|
||||
|
||||
// delete partitions
|
||||
val recordsToDelete = df1.filter(entry => {
|
||||
val partitionPath : String = entry.getString(1)
|
||||
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) || partitionPath.equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)
|
||||
})
|
||||
val updatedParams = fooTableParams.updated(DataSourceWriteOptions.OPERATION.key(), WriteOperationType.DELETE_PARTITION.name())
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, updatedParams, recordsToDelete)
|
||||
if ( usePartitionsToDeleteConfig) {
|
||||
fooTableParams.updated(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
}
|
||||
// delete partitions contains the primary key
|
||||
val recordsToDelete = df1.filter(entry => {
|
||||
val partitionPath : String = entry.getString(1)
|
||||
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) ||
|
||||
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)
|
||||
})
|
||||
val updatedParams = fooTableParams.updated(DataSourceWriteOptions.OPERATION.key(), WriteOperationType.DELETE_PARTITION.name())
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, updatedParams, recordsToDelete)
|
||||
|
||||
val snapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(0, snapshotDF3.filter(entry => {
|
||||
val partitionPath = entry.getString(3)
|
||||
!partitionPath.equals(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
}).count())
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
val snapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.load(path.toAbsolutePath.toString + "/*/*/*/*")
|
||||
assertEquals(0, snapshotDF3.filter(entry => {
|
||||
val partitionPath = entry.getString(3)
|
||||
!partitionPath.equals(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
}).count())
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
def dropMetaFields(df: Dataset[Row]) : Dataset[Row] = {
|
||||
df.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
|
||||
Reference in New Issue
Block a user