1
0

feat(SparkDataSource): add additional feature to drop later arriving dups

This commit is contained in:
jiale.tan
2018-10-04 17:56:51 -07:00
committed by vinoth chandar
parent 8485b9e263
commit 1628d044ac
5 changed files with 170 additions and 70 deletions

View File

@@ -139,4 +139,47 @@ class DataSourceTest extends AssertionsForJUnit {
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie").load(basePath + "/*/*/*/*")
assertEquals(100, hoodieROViewDF1.count()) // still 100, since we only updated
}
@Test def testDropInsertDup(): Unit = {
val insert1Cnt = 10
val insert2DupKeyCnt = 9
val insert2NewKeyCnt = 2
val totalUniqueKeyToGenerate = insert1Cnt + insert2NewKeyCnt
val allRecords = dataGen.generateInserts("001", totalUniqueKeyToGenerate)
val inserts1 = allRecords.subList(0, insert1Cnt)
val inserts2New = dataGen.generateSameKeyInserts("002", allRecords.subList(insert1Cnt, insert1Cnt + insert2NewKeyCnt))
val inserts2Dup = dataGen.generateSameKeyInserts("002", inserts1.subList(0, insert2DupKeyCnt))
val records1 = DataSourceTestUtils.convertToStringList(inserts1).toList
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("com.uber.hoodie")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Overwrite)
.save(basePath)
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie")
.load(basePath + "/*/*/*/*")
assertEquals(insert1Cnt, hoodieROViewDF1.count())
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
val records2 = DataSourceTestUtils
.convertToStringList(inserts2Dup ++ inserts2New)
.toList
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("com.uber.hoodie")
.options(commonOpts)
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true")
.mode(SaveMode.Append)
.save(basePath)
val hoodieROViewDF2 = spark.read.format("com.uber.hoodie")
.load(basePath + "/*/*/*/*")
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
val hoodieIncViewDF2 = spark.read.format("com.uber.hoodie")
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
.load(basePath)
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
}
}