Skip to content

Commit 3d9e1da

Browse files
jerryyueXuQianJin-Stars
authored andcommitted
optimize multiple partial update code
1 parent 2ebd741 commit 3d9e1da

File tree

25 files changed

+846
-421
lines changed

25 files changed

+846
-421
lines changed

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ public HoodieWriteMetadata<O> write(String instantTime,
4343
try {
4444
// De-dupe/merge if needed
4545
I dedupedRecords =
46-
combineOnCondition(shouldCombine, inputRecords, shuffleParallelism, table, executor.config.getSchema());
46+
combineOnCondition(shouldCombine, inputRecords, shuffleParallelism, table);
4747

4848
Instant lookupBegin = Instant.now();
4949
I taggedRecords = dedupedRecords;
@@ -69,8 +69,8 @@ protected abstract I tag(
6969
I dedupedRecords, HoodieEngineContext context, HoodieTable<T, I, K, O> table);
7070

7171
public I combineOnCondition(
72-
boolean condition, I records, int parallelism, HoodieTable<T, I, K, O> table, String schemaString) {
73-
return condition ? deduplicateRecords(records, table, parallelism, schemaString) : records;
72+
boolean condition, I records, int parallelism, HoodieTable<T, I, K, O> table) {
73+
return condition ? deduplicateRecords(records, table, parallelism) : records;
7474
}
7575

7676
/**
@@ -81,8 +81,8 @@ public I combineOnCondition(
8181
* @return Collection of HoodieRecord already be deduplicated
8282
*/
8383
public I deduplicateRecords(
84-
I records, HoodieTable<T, I, K, O> table, int parallelism, String schemaString) {
85-
return deduplicateRecords(records, table.getIndex(), parallelism, schemaString);
84+
I records, HoodieTable<T, I, K, O> table, int parallelism) {
85+
return deduplicateRecords(records, table.getIndex(), parallelism, table.getConfig().getSchema());
8686
}
8787

8888
public abstract I deduplicateRecords(

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.hudi.table.action.commit;
2020

2121
import org.apache.hudi.client.WriteStatus;
22+
import org.apache.hudi.common.config.SerializableSchema;
2223
import org.apache.hudi.common.data.HoodieData;
2324
import org.apache.hudi.common.engine.HoodieEngineContext;
2425
import org.apache.hudi.common.model.HoodieAvroRecord;
@@ -55,16 +56,15 @@ protected HoodieData<HoodieRecord<T>> tag(HoodieData<HoodieRecord<T>> dedupedRec
5556
public HoodieData<HoodieRecord<T>> deduplicateRecords(
5657
HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaString) {
5758
boolean isIndexingGlobal = index.isGlobal();
58-
Properties properties = new Properties();
59-
properties.put("schema", schemaString);
59+
final SerializableSchema schema = new SerializableSchema(schemaString);
6060
return records.mapToPair(record -> {
6161
HoodieKey hoodieKey = record.getKey();
6262
// If index used is global, then records are expected to differ in their partitionPath
6363
Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
6464
return Pair.of(key, record);
6565
}).reduceByKey((rec1, rec2) -> {
6666
@SuppressWarnings("unchecked")
67-
T reducedData = (T) rec2.getData().preCombine(rec1.getData(), properties);
67+
T reducedData = (T) rec2.getData().preCombine(rec1.getData(), schema.get(), new Properties());
6868
HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey();
6969

7070
return new HoodieAvroRecord<>(reducedKey, reducedData);

hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
package org.apache.hudi.table.action.commit;
2020

21+
import org.apache.avro.Schema;
2122
import org.apache.hudi.client.WriteStatus;
2223
import org.apache.hudi.common.data.HoodieListData;
2324
import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -98,9 +99,8 @@ public List<HoodieRecord<T>> deduplicateRecords(
9899
final T data1 = rec1.getData();
99100
final T data2 = rec2.getData();
100101

101-
Properties properties = new Properties();
102-
properties.put("schema", schemaString);
103-
@SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1, properties);
102+
final Schema schema = new Schema.Parser().parse(schemaString);
103+
@SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1, schema, new Properties());
104104
// we cannot allow the user to change the key or partitionPath, since that will affect
105105
// everything
106106
// so pick it from one of the records.

hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/table/action/commit/FlinkWriteHelperTest.java

Lines changed: 0 additions & 112 deletions
This file was deleted.

hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -103,7 +103,7 @@ public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> inputRecords,
103103

104104
if (performDedupe) {
105105
dedupedRecords = (List<HoodieRecord<T>>) JavaWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
106-
parallelism, table, config.getSchema());
106+
parallelism, table);
107107
}
108108

109109
final List<HoodieRecord<T>> repartitionedRecords = (List<HoodieRecord<T>>) partitioner.repartitionRecords(dedupedRecords, parallelism);

hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -100,7 +100,7 @@ public HoodieData<WriteStatus> bulkInsert(HoodieData<HoodieRecord<T>> inputRecor
100100

101101
if (performDedupe) {
102102
dedupedRecords = (HoodieData<HoodieRecord<T>>) HoodieWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
103-
parallelism, table, config.getSchema());
103+
parallelism, table);
104104
}
105105

106106
// only JavaRDD is supported for Spark partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463

hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java

Lines changed: 12 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -18,18 +18,6 @@
1818

1919
package org.apache.hudi.avro;
2020

21-
import org.apache.hudi.common.config.SerializableSchema;
22-
import org.apache.hudi.common.model.HoodieOperation;
23-
import org.apache.hudi.common.model.HoodieRecord;
24-
import org.apache.hudi.common.model.HoodieRecordPayload;
25-
import org.apache.hudi.common.model.MultipleOrderingVal2ColsInfo;
26-
import org.apache.hudi.common.util.Option;
27-
import org.apache.hudi.common.util.StringUtils;
28-
import org.apache.hudi.common.util.collection.Pair;
29-
import org.apache.hudi.exception.HoodieException;
30-
import org.apache.hudi.exception.HoodieIOException;
31-
import org.apache.hudi.exception.SchemaCompatibilityException;
32-
3321
import org.apache.avro.AvroRuntimeException;
3422
import org.apache.avro.Conversions;
3523
import org.apache.avro.Conversions.DecimalConversion;
@@ -54,6 +42,16 @@
5442
import org.apache.avro.io.JsonDecoder;
5543
import org.apache.avro.io.JsonEncoder;
5644
import org.apache.avro.specific.SpecificRecordBase;
45+
import org.apache.hudi.common.config.SerializableSchema;
46+
import org.apache.hudi.common.model.HoodieOperation;
47+
import org.apache.hudi.common.model.HoodieRecord;
48+
import org.apache.hudi.common.model.HoodieRecordPayload;
49+
import org.apache.hudi.common.util.Option;
50+
import org.apache.hudi.common.util.StringUtils;
51+
import org.apache.hudi.common.util.collection.Pair;
52+
import org.apache.hudi.exception.HoodieException;
53+
import org.apache.hudi.exception.HoodieIOException;
54+
import org.apache.hudi.exception.SchemaCompatibilityException;
5755

5856
import java.io.ByteArrayInputStream;
5957
import java.io.ByteArrayOutputStream;
@@ -70,13 +68,12 @@
7068
import java.util.ArrayList;
7169
import java.util.Collection;
7270
import java.util.Collections;
71+
import java.util.Deque;
7372
import java.util.HashMap;
7473
import java.util.Iterator;
74+
import java.util.LinkedList;
7575
import java.util.List;
7676
import java.util.Map;
77-
import java.util.Deque;
78-
import java.util.LinkedList;
79-
import java.util.Objects;
8077
import java.util.TimeZone;
8178
import java.util.stream.Collectors;
8279

@@ -497,17 +494,6 @@ public static Object getNestedFieldValAsString(GenericRecord record, String fiel
497494
return obj == null ? "" : StringUtils.objToString(obj);
498495
}
499496

500-
public static Object getMultipleNestedFieldVals(GenericRecord record, String fieldMappings, boolean consistentLogicalTimestampEnabled) {
501-
MultipleOrderingVal2ColsInfo multipleOrderingVal2ColsInfo = new MultipleOrderingVal2ColsInfo(fieldMappings);
502-
multipleOrderingVal2ColsInfo.getOrderingVal2ColsInfoList().forEach(orderingVal2ColsInfo -> {
503-
Object val = getNestedFieldVal(record, orderingVal2ColsInfo.getOrderingField(), true, consistentLogicalTimestampEnabled);
504-
if (Objects.nonNull(val)) {
505-
orderingVal2ColsInfo.setOrderingValue(val.toString());
506-
}
507-
});
508-
return multipleOrderingVal2ColsInfo.generateOrderingText();
509-
}
510-
511497
/**
512498
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
513499
*/

hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,10 @@ public SerializableSchema() {
3838
public SerializableSchema(Schema schema) {
3939
this.schema = newCopy(schema);
4040
}
41+
42+
public SerializableSchema(String schemaStr) {
43+
this.schema = new Schema.Parser().parse(schemaStr);
44+
}
4145

4246
public SerializableSchema(SerializableSchema serializableSchema) {
4347
this(serializableSchema.schema);

hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,20 @@ default T preCombine(T oldValue, Properties properties) {
5858
return preCombine(oldValue);
5959
}
6060

61+
/**
62+
* When more than one HoodieRecord have the same HoodieKey in the incoming batch, this function combines them before attempting to insert/upsert by taking in a schema.
63+
* Implementation can leverage the schema to decide their business logic to do preCombine.
64+
*
65+
* @param oldValue instance of the old {@link HoodieRecordPayload} to be combined with.
66+
* @param schema Payload related schema. For example use schema to overwrite old instance for specified fields that doesn't equal to default value.
67+
* @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage.
68+
* @return the combined value
69+
*/
70+
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
71+
default T preCombine(T oldValue, Schema schema, Properties properties) {
72+
return preCombine(oldValue, properties);
73+
}
74+
6175
/**
6276
* This methods is deprecated. Please refer to {@link #combineAndGetUpdateValue(IndexedRecord, Schema, Properties)} for java docs.
6377
*/

0 commit comments

Comments
 (0)