DeduplicateRecords based on recordKey if global index is used
This commit is contained in:
committed by
vinoth chandar
parent
123da020e2
commit
291a88ba94
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||
@@ -111,10 +112,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
boolean rollbackInFlight) {
|
||||
this(jsc, clientConfig, rollbackInFlight, HoodieIndex.createIndex(clientConfig, jsc));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
boolean rollbackInFlight, HoodieIndex index) {
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
|
||||
this.jsc = jsc;
|
||||
this.config = clientConfig;
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
this.index = index;
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
|
||||
if (rollbackInFlight) {
|
||||
@@ -1051,10 +1058,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
/**
|
||||
* Deduplicate Hoodie records, using the given deduplication funciton.
|
||||
*/
|
||||
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
return records
|
||||
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
|
||||
.mapToPair(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
// If index used is global, then records are expected to differ in their partitionPath
|
||||
Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
|
||||
return new Tuple2<>(key, record);
|
||||
})
|
||||
.reduceByKey((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
|
||||
Reference in New Issue
Block a user