Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
Expand Down Expand Up @@ -83,9 +84,9 @@ public I combineOnCondition(
*/
public I deduplicateRecords(
I records, HoodieTable<T, I, K, O> table, int parallelism) {
return deduplicateRecords(records, table.getIndex(), parallelism);
return deduplicateRecords(records, table.getIndex(), table.getConfig(), parallelism);
}

public abstract I deduplicateRecords(
I records, HoodieIndex<T, I, K, O> index, int parallelism);
I records, HoodieIndex<T, I, K, O> index, HoodieWriteConfig writeConfig, int parallelism);
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,13 @@
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;

import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaRDD;

import scala.Serializable;
import scala.Tuple2;

/**
Expand All @@ -49,21 +52,48 @@ public static SparkWriteHelper newInstance() {
@Override
public JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index,
int parallelism) {
boolean isIndexingGlobal = index.isGlobal();
return records.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());
// we cannot allow the user to change the key or partitionPath, since that will affect
// everything
// so pick it from one of the records.
return new HoodieRecord<T>(rec1.getKey(), reducedData);
}, parallelism).map(Tuple2::_2);
HoodieWriteConfig writeConfig, int parallelism) {
return new RecordDeduper<T>(index.isGlobal(), writeConfig.getSchema(), parallelism).deduplicateRecords(records);
}

/**
* Helper class to dedupe records.
* @param <T>
*/
private static class RecordDeduper<T extends HoodieRecordPayload> implements Serializable {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made this to a class to take advantatage of lazy creation of Schema in executors.


private transient Schema schema;
private final String schemaStr;
private final boolean isIndexingGlobal;
private final int parallelism;

public RecordDeduper(boolean isIndexingGlobal, String schemaStr, int parallelism) {
this.isIndexingGlobal = isIndexingGlobal;
this.parallelism = parallelism;
this.schemaStr = schemaStr;
}

private Schema getSchema() {
if (null == schema) {
schema = Schema.parse(schemaStr);
}
return schema;
}

public JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records) {
return records.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(), getSchema());
// we cannot allow the user to change the key or partitionPath, since that will affect
// everything
// so pick it from one of the records.
return new HoodieRecord<T>(rec1.getKey(), reducedData);
}, parallelism).map(Tuple2::_2);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -236,17 +236,23 @@ private void testDeduplication(
JavaRDD<HoodieRecord<RawTripTestPayload>> records =
jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);

String schemaStr = "{\"namespace\": \"org.apache.hudi.avro.model\", \"type\": \"record\", " +
"\"name\": \"HoodieInstantInfo\", \"fields\": [{\"name\": \"commitTime\", \"type\": \"string\"}]}";
// Global dedup should be done based on recordKey only
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(true);
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect();
HoodieWriteConfig writeConfig = mock(HoodieWriteConfig.class);
when(writeConfig.getSchema()).thenReturn(schemaStr);
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, writeConfig, 1).collect();
assertEquals(1, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);

// non-Global dedup should be done based on both recordKey and partitionPath
index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(false);
dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect();
writeConfig = mock(HoodieWriteConfig.class);
when(writeConfig.getSchema()).thenReturn(schemaStr);
dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, writeConfig, 1).collect();
assertEquals(2, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,9 +41,23 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri
* When more than one HoodieRecord have the same HoodieKey, this function combines them before attempting to
* insert/upsert (if combining turned on in HoodieClientConfig).
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
@Deprecated
T preCombine(T another);

/**
* When more than one HoodieRecord have the same HoodieKey, this function combines them before attempting to
* insert/upsert (if combining turned on in HoodieClientConfig).
* Any custom Record payload implementation that requires schema must override this method.
*
* @param another Record payload to combine with.
* @param schema Schema used for writing to dataset.
* @return Combined Record.
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
default T preCombine(T another, Schema schema) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we just introduce a config that simply uses combineAndGetUpdateValue() to perform the preCombine as well. We have debated doing this for a while. Do you see a reason not to do something like that.

return preCombine(another);
}

/**
* This methods lets you write custom merging/combining logic to produce new values as a function of current value on
* storage and whats contained in this object.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoo
if (records.containsKey(key)) {
// Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be
// done when a delete (empty payload) is encountered before or after an insert/update.
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(records.get(key).getData());
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(records.get(key).getData(), readerSchema);
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
} else {
// Put the record as is
Expand Down