-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-2815] add partial overwrite payload to support partial overwrit… #4724
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
d73d7f5
6b6a60f
2fa2d57
21df6fe
940f6de
14edef0
ce561bc
c6f524e
d3b3e05
10e080b
a86b7ff
d11c670
715d4b0
e89fd60
3f771d3
b823e94
9c15335
eef40bc
3a373c2
6fbf453
1e236ba
ac3e72a
f1a8d0c
3697d8c
975c463
46ea95d
4aaee39
6a13069
466a633
fe4aefd
0b9f295
6731992
4b975fd
7a30b08
d6e38af
ef9ff1a
dd7e772
8a4cfb7
6af6076
77b0f3f
6bb4181
6c4b714
b851feb
4d86424
0b21be2
55f5626
d2aed60
4c15551
b9230e0
1e68d6f
f28bad6
6fa32a0
da9962b
f52553b
a5b9f66
69f058c
2a18375
8cba0a9
ced2def
1409c0b
cd47bc9
d22d93f
d0d6981
180b690
b4770df
f76144b
f7886f8
fc6c7a7
7d89404
801c69d
cf03735
d963079
5f59bcb
04baf70
5da95d5
a51bdb5
9e1cad8
5403db3
151ce1e
ff16cdc
54808ec
6530d83
6570198
967b336
399eb8d
07d6929
f9ae271
31b54c7
95ef13c
1a7157a
145440b
c0eecb5
035c3ca
ece2ae6
a55ce33
895becc
00b2e45
4512e96
c163ac2
402f60e
b825b8a
029622b
931747d
75abad6
9037045
9c40d0c
d9ca8e1
dfc05b7
618fe26
b28f5d2
542cec6
75056ea
d1e31f8
e19b5d1
b709f75
1ce9a5e
dcbb074
0640f20
d482527
7f5ee51
5558b79
a9b4110
ffac31e
5854243
f8092a3
32b9700
27adaa2
9c49e43
1959d8b
4568fae
c43747e
5b66abf
06ac8cb
24cc379
5a0a1e9
d9da263
20b1ee4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -58,6 +58,31 @@ default T preCombine(T oldValue, Properties properties) { | |
| return preCombine(oldValue); | ||
| } | ||
|
|
||
| /** | ||
| *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 property map. | ||
| * | ||
| * @param oldValue instance of the old {@link HoodieRecordPayload} to be combined with. | ||
| * @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage. | ||
| * @param schema Schema used for record | ||
| * @return the combined value | ||
| */ | ||
| @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) | ||
| default T preCombine(T oldValue, Properties properties, Schema schema) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think currently established semantic for
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks @alexeykudinkin for reviewing this. What we are trying to do is implement partial update purpose. For example, let's assume the record schema is Actually, what we want to implement is similar with
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Right, that's exactly my question: why do you want to implement such semantic w/in Essentially with this change you will introduce a way for 2 records w/in the batch to be combined into 1. But why do you need this? After all you can achieve the same goal if you just stop de-duping your records, and then subsequently merge them against what is on disk
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @alexeykudinkin, I got your point. if we have to combine two records to a combined one, we'd better to implement the combine logics in other place, maybe in some Here are some options from mine that
That's what my thought here, and I'm glad to listen your useful suggestions. :)
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let me try to clarify a few things:
Please see my comment regarding
You're bringing up a good points, let's dive into them one by one: so currently we have 2 mechanisms
You rightfully mention some of the invariants are currently that the batch would be de-duped at certain level (b/c we have to maintain PK uniqueness on disk), and so we might need to shift that to accommodate for case that you have. And that's exactly what my question was: if you can elaborate on use-case that you have at hand that you're trying to solve w/ this PR, i would be able to better understand where you're coming from and what's the best path forward for us here. Questions i'm looking an answers for are basically following:
and etc. I'm happy to set some 30min to talk in person regarding this or connect on Slack and discuss it there.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @alexeykudinkin , Thanks a lot for you detail clarification.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. BTW, thanks a lot for you time, will ping you on slack.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. CC @rmahindra123 who encountered a necessity to do preCombine but to combine bits and pieces from both records to return a new one. Rajesh: do you wanna go over your use-case may be.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @alexeykudinkin , I'm sorry that I still don't find a suitable time to align with online, may i check any thoughts or suggests on this PR? |
||
| return preCombine(oldValue, properties); | ||
| } | ||
|
|
||
| /** | ||
| * When more than one HoodieRecord have the same HoodieKey in the incoming batch, and get the merged result after calling preCombine method instead of choose one of two records, | ||
| * can call this method to get the order among combined record with previous records | ||
| * @param oldValue instance of the old {@link HoodieRecordPayload} to be compare. | ||
| * @return a negative integer, zero, or a positive integer as this object is less than, equal to, or greater than the specified object. | ||
| * | ||
| */ | ||
| @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) | ||
| default int compareTo(T oldValue) { | ||
| return 0; | ||
| } | ||
|
|
||
| /** | ||
| * This methods is deprecated. Please refer to {@link #combineAndGetUpdateValue(IndexedRecord, Schema, Properties)} for java docs. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,133 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hudi.common.model; | ||
|
|
||
| import org.apache.hudi.common.util.Option; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.avro.generic.IndexedRecord; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Objects; | ||
| import java.util.Properties; | ||
| import java.util.function.BiFunction; | ||
|
|
||
| import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro; | ||
|
|
||
| /** | ||
| * The only difference with {@link OverwriteNonDefaultsWithLatestAvroPayload} is that it supports | ||
| * merging the latest non-null partial fields with the old record instead of replacing the whole record. | ||
| * And merging the non-null fields during preCombine multiple records with same record key instead of choosing the latest record based on ordering field. | ||
| * | ||
| * <p> Regarding #combineAndGetUpdateValue, Assuming a {@link GenericRecord} has row schema: (f0 int , f1 int, f2 int). | ||
| * The first record value is: (1, 2, 3), the second record value is: (4, 5, null) with the field f2 value as null. | ||
| * Calling the #combineAndGetUpdateValue method of the two records returns record: (4, 5, 3). | ||
| * Note that field f2 value is ignored because it is null. </p> | ||
| * | ||
| * <p> Regarding #preCombine, Assuming a {@link GenericRecord} has row schema: (f0 int , f1 int, f2 int, o1 int), | ||
| * and initial two {@link PartialOverwriteWithLatestAvroPayload} with different ordering value. | ||
| * The first record value is (1, null, 1, 1) with the filed f1 value as null, the second value is: (2, 2, null, 2) with the f2 value as null. | ||
| * Calling the #preCombine method of the two records returns record: (2, 2, 1, 2). | ||
| * Note: | ||
| * <ol> | ||
| * <li>the field f0 value is 2 because the ordering value of second record is bigger.</li> | ||
| * <li>the filed f1 value is 2 because the f2 value of first record is null.</li> | ||
| * <li>the filed f2 value is 1 because the f2 value of second record is null.</li> | ||
| * <li>the filed o1 value is 2 because the ordering value of second record is bigger.</li> | ||
| * </ol> | ||
| * | ||
| * </p> | ||
| */ | ||
| public class PartialOverwriteWithLatestAvroPayload extends OverwriteWithLatestAvroPayload { | ||
|
|
||
| public PartialOverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) { | ||
| super(record, orderingVal); | ||
| } | ||
|
|
||
| public PartialOverwriteWithLatestAvroPayload(Option<GenericRecord> record) { | ||
| super(record); // natural order | ||
| } | ||
|
|
||
| @Override | ||
| public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { | ||
| if (recordBytes.length == 0) { | ||
| return Option.empty(); | ||
| } | ||
|
|
||
| GenericRecord incomingRecord = bytesToAvro(recordBytes, schema); | ||
| GenericRecord currentRecord = (GenericRecord) currentValue; | ||
| if (isDeleteRecord(incomingRecord)) { | ||
| return Option.empty(); | ||
| } | ||
| return Option.of(overwriteWithNonNullValue(schema, currentRecord, incomingRecord)); | ||
| } | ||
|
|
||
| @Override | ||
| public int compareTo(OverwriteWithLatestAvroPayload oldValue) { | ||
| return this.orderingVal.compareTo(oldValue.orderingVal); | ||
| } | ||
|
|
||
| @Override | ||
| public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload oldValue, Properties properties, Schema schema) { | ||
| if (null == schema) { | ||
| // using default preCombine logic | ||
| return super.preCombine(oldValue); | ||
| } | ||
|
|
||
| try { | ||
| Option<IndexedRecord> incomingOption = this.getInsertValue(schema); | ||
| Option<IndexedRecord> oldRecordOption = oldValue.getInsertValue(schema); | ||
|
|
||
| if (incomingOption.isPresent() && oldRecordOption.isPresent()) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In general it's better to express common functionality in a way that would allow it to be re-used and adopted in other places: here for ex, we can reuse the same routine of combining 2 records into one, across 2 methods if we properly abstract it
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. abstracted the merge method, but still in current class. |
||
| boolean inComingRecordIsLatest = this.compareTo(oldValue) >= 0; | ||
| // ordering two records by ordering value | ||
| GenericRecord firstRecord = (GenericRecord) (inComingRecordIsLatest ? oldRecordOption.get() : incomingOption.get()); | ||
| GenericRecord secondRecord = (GenericRecord) (inComingRecordIsLatest ? incomingOption.get() : oldRecordOption.get()); | ||
| GenericRecord mergedRecord = overwriteWithNonNullValue(schema, firstRecord, secondRecord); | ||
| return new PartialOverwriteWithLatestAvroPayload(mergedRecord, inComingRecordIsLatest ? this.orderingVal : oldValue.orderingVal); | ||
| } else { | ||
| return super.preCombine(oldValue); | ||
| } | ||
| } catch (IOException e) { | ||
| return super.preCombine(oldValue); | ||
| } | ||
| } | ||
|
|
||
| private GenericRecord mergeRecord(Schema schema, GenericRecord first, GenericRecord second, BiFunction<Object, Object, Object> mergeFunc) { | ||
| schema.getFields().forEach(field -> { | ||
| Object firstValue = first.get(field.name()); | ||
| Object secondValue = second.get(field.name()); | ||
| first.put(field.name(), mergeFunc.apply(firstValue, secondValue)); | ||
| }); | ||
| return first; | ||
| } | ||
|
|
||
| /** | ||
| * Merge two records, the merged value of each filed will adopt the filed value from secondRecord if the value is not null, otherwise, adopt the filed value from firstRecord. | ||
| * | ||
| * @param schema record schema to loop fields | ||
| * @param firstRecord the base record need to be updated | ||
| * @param secondRecord the new record provide new field value | ||
| * @return merged records | ||
| */ | ||
| private GenericRecord overwriteWithNonNullValue(Schema schema, GenericRecord firstRecord, GenericRecord secondRecord) { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I really liked the idea of having a record payload that does partial merging. However, if I understood it correctly, what's proposed here is to do so in a very specific way: you're favoring the income record field's values, unless they are null (in which case, you would keep the existing one). I'm not saying this is not valuable, but that the idea of doing partial merging is so good that maybe we could have something more generic. I'm going to suggest a few changes in order to accomplish that:
It's just an idea, that could make what you proposed useful for many more use cases. Hope this made sense, and thanks for bringing this idea!
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @alvarolemos , thanks a lot for your useful suggestion. Yeah, I also considered to abstract the merge logic by using an abstract
|
||
| return mergeRecord(schema, firstRecord, secondRecord, (first, second) -> Objects.isNull(second) ? first : second); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why we must need a
compareTohere ?Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The previous logic of
data2.preCombine(data1)is that return one of data1 or data2 ordering by theirorderVal. But if we merge/combine data1 and data2 into a new payload(reduceData), thedata1.equals(reduceData)is always false. In order to get theHoodieKeyandHoodieOperationfor new HoodieRecord withreduceData, we need to get the latestHoodieKeyandHoodieOperationfromdata1anddata2,compareTois used for replace#preCombineto compare theirorderingVal.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually,
rec1andrec2should have same HoodieKey here, right, but the HodieOperation might different.