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

@@ -196,6 +196,16 @@ public class HoodieTestDataGenerator {
return inserts; return inserts;
} }
public List<HoodieRecord> generateSameKeyInserts(String commitTime, List<HoodieRecord> origin) throws IOException {
List<HoodieRecord> copy = new ArrayList<>();
for (HoodieRecord r: origin) {
HoodieKey key = r.getKey();
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
copy.add(record);
}
return copy;
}
public List<HoodieRecord> generateDeletes(String commitTime, Integer n) throws IOException { public List<HoodieRecord> generateDeletes(String commitTime, Integer n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n); List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts); return generateDeletesFromExistingRecords(inserts);

View File

@@ -158,4 +158,15 @@ public class DataSourceUtils {
return incomingHoodieRecords; return incomingHoodieRecords;
} }
} }
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
JavaRDD<HoodieRecord> incomingHoodieRecords,
Map<String, String> parameters) throws Exception {
HoodieWriteConfig writeConfig = HoodieWriteConfig
.newBuilder()
.withPath(parameters.get("path"))
.withProps(parameters).build();
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
}
} }

View File

@@ -145,6 +145,13 @@ object DataSourceWriteOptions {
val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix" val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix"
val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_" val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_"
/**
* Flag to indicate whether to drop duplicates upon insert.
* By default insert will accept duplicates, to gain extra performance.
*/
val INSERT_DROP_DUPS_OPT_KEY = "hoodie.datasource.write.insert.drop.duplicates"
val DEFAULT_INSERT_DROP_DUPS_OPT_VAL = "false"
// HIVE SYNC SPECIFIC CONFIGS // HIVE SYNC SPECIFIC CONFIGS
//NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes //NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
// unexpected issues with config getting reset // unexpected issues with config getting reset

View File

@@ -122,6 +122,7 @@ class DefaultSource extends RelationProvider
defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL) defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL)
defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL) defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL)
defaultsMap.putIfAbsent(INSERT_DROP_DUPS_OPT_KEY, DEFAULT_INSERT_DROP_DUPS_OPT_VAL)
defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL) defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL)
defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL) defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL)
defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL) defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL)
@@ -159,7 +160,21 @@ class DefaultSource extends RelationProvider
} }
val storageType = parameters(STORAGE_TYPE_OPT_KEY) val storageType = parameters(STORAGE_TYPE_OPT_KEY)
val operation = parameters(OPERATION_OPT_KEY) val operation =
// It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true
// Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly
// or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) .
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean &&
parameters(OPERATION_OPT_KEY) == UPSERT_OPERATION_OPT_VAL) {
log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " +
s"when $INSERT_DROP_DUPS_OPT_KEY is set to be true, " +
s"overriding the $OPERATION_OPT_KEY to be $INSERT_OPERATION_OPT_VAL")
INSERT_OPERATION_OPT_VAL
} else {
parameters(OPERATION_OPT_KEY)
}
// register classes & schemas // register classes & schemas
val structName = s"${tblName.get}_record" val structName = s"${tblName.get}_record"
@@ -177,14 +192,26 @@ class DefaultSource extends RelationProvider
toProperties(parameters) toProperties(parameters)
) )
val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace)
val hoodieRecords = genericRecords.map(gr => { val hoodieAllIncomingRecords = genericRecords.map(gr => {
val orderingVal = DataSourceUtils.getNestedFieldValAsString( val orderingVal = DataSourceUtils.getNestedFieldValAsString(
gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]] gr, parameters(PRECOMBINE_FIELD_OPT_KEY)).asInstanceOf[Comparable[_]]
DataSourceUtils.createHoodieRecord(gr, DataSourceUtils.createHoodieRecord(gr,
orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY)) orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY))
}).toJavaRDD(); }).toJavaRDD();
val jsc = new JavaSparkContext(sparkContext)
val hoodieRecords =
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
DataSourceUtils.dropDuplicates(
jsc,
hoodieAllIncomingRecords,
mapAsJavaMap(parameters))
} else {
hoodieAllIncomingRecords
}
if (!hoodieRecords.isEmpty()) {
val basePath = new Path(parameters.get("path").get) val basePath = new Path(parameters.get("path").get)
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
var exists = fs.exists(basePath) var exists = fs.exists(basePath)
@@ -210,7 +237,6 @@ class DefaultSource extends RelationProvider
} }
// Create a HoodieWriteClient & issue the write. // Create a HoodieWriteClient & issue the write.
val jsc = new JavaSparkContext(sparkContext);
val client = DataSourceUtils.createHoodieClient(jsc, val client = DataSourceUtils.createHoodieClient(jsc,
schema.toString, schema.toString,
path.get, path.get,
@@ -248,7 +274,7 @@ class DefaultSource extends RelationProvider
} }
client.close client.close
} else { } else {
log.error(s"Upsert failed with ${errorCount} errors :"); log.error(s"$operation failed with ${errorCount} errors :");
if (log.isTraceEnabled) { if (log.isTraceEnabled) {
log.trace("Printing out the top 100 errors") log.trace("Printing out the top 100 errors")
writeStatuses.rdd.filter(ws => ws.hasErrors) writeStatuses.rdd.filter(ws => ws.hasErrors)
@@ -262,6 +288,9 @@ class DefaultSource extends RelationProvider
}) })
} }
} }
} else {
log.info("new batch has no new records, skipping...")
}
createRelation(sqlContext, parameters, df.schema) createRelation(sqlContext, parameters, df.schema)
} }

View File

@@ -139,4 +139,47 @@ class DataSourceTest extends AssertionsForJUnit {
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie").load(basePath + "/*/*/*/*") val hoodieROViewDF1 = spark.read.format("com.uber.hoodie").load(basePath + "/*/*/*/*")
assertEquals(100, hoodieROViewDF1.count()) // still 100, since we only updated 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)
}
} }