1
0

[HUDI-3001] Clean up the marker directory when finish bootstrap operation. (#4298)

This commit is contained in:
xiarixiaoyao
2021-12-17 04:36:01 +08:00
committed by GitHub
parent a8a192aef5
commit 294d712948
2 changed files with 11 additions and 2 deletions

View File

@@ -17,7 +17,7 @@
package org.apache.hudi.functional
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector
import org.apache.hudi.common.fs.FSUtils
@@ -31,9 +31,9 @@ import org.apache.spark.sql.{SaveMode, SparkSession}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.io.TempDir
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import java.time.Instant
import java.util.Collections
import scala.collection.JavaConverters._
class TestDataSourceForBootstrap {
@@ -106,6 +106,8 @@ class TestDataSourceForBootstrap {
DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
extraOpts = Map(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
)
// check marked directory clean up
assert(!fs.exists(new Path(basePath, ".hoodie/.temp/00000000000001")))
// Read bootstrapped table and verify count
var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
@@ -161,6 +163,9 @@ class TestDataSourceForBootstrap {
Some("datestr"),
Map(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true"))
// check marked directory clean up
assert(!fs.exists(new Path(basePath, ".hoodie/.temp/00000000000001")))
// Read bootstrapped table and verify count
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
assertEquals(numRecords, hoodieROViewDF1.count())