-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-115] Enhance OverwriteWithLatestAvroPayload to also respect ordering value of record in storage #1704
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
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 |
|---|---|---|
|
|
@@ -50,8 +50,25 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri | |
| * @param schema Schema used for record | ||
| * @return new combined/merged value to be written back to storage. EMPTY to skip writing this record. | ||
| */ | ||
| @Deprecated | ||
| Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException; | ||
|
|
||
| /** | ||
| * 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. | ||
| * <p> | ||
| * eg: 1) You are updating counters, you may want to add counts to currentValue and write back updated counts 2) You | ||
| * may be reading DB redo logs, and merge them with current image for a database row on storage | ||
| * | ||
| * @param currentValue Current value in storage, to merge/combine this payload with | ||
| * @param schema Schema used for record | ||
| * @param props Payload related properties. For example pass the ordering field(s) name to extract from value in storage. | ||
| * @return new combined/merged value to be written back to storage. EMPTY to skip writing this record. | ||
| */ | ||
| default Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Map<String, String> props) throws IOException { | ||
|
||
| return combineAndGetUpdateValue(currentValue, schema); | ||
| } | ||
|
|
||
| /** | ||
| * Generates an avro record out of the given HoodieRecordPayload, to be written out to storage. Called when writing a | ||
| * new value for the given HoodieKey, wherein there is no existing record in storage to be combined against. (i.e | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,7 +18,10 @@ | |
|
|
||
| package org.apache.hudi.common.model; | ||
|
|
||
| import org.apache.hudi.avro.HoodieAvroUtils; | ||
| import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro; | ||
| import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldVal; | ||
|
|
||
| import java.util.Map; | ||
| import org.apache.hudi.common.util.Option; | ||
|
|
||
| import org.apache.avro.Schema; | ||
|
|
@@ -67,7 +70,7 @@ public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException { | |
| if (recordBytes.length == 0) { | ||
| return Option.empty(); | ||
| } | ||
| IndexedRecord indexedRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); | ||
| IndexedRecord indexedRecord = bytesToAvro(recordBytes, schema); | ||
| if (isDeleteRecord((GenericRecord) indexedRecord)) { | ||
| return Option.empty(); | ||
| } else { | ||
|
|
@@ -83,4 +86,39 @@ private boolean isDeleteRecord(GenericRecord genericRecord) { | |
| Object deleteMarker = genericRecord.get("_hoodie_is_deleted"); | ||
| return (deleteMarker instanceof Boolean && (boolean) deleteMarker); | ||
| } | ||
|
|
||
| @Override | ||
| public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Map<String, String> props) throws IOException { | ||
| if (recordBytes.length == 0) { | ||
| return Option.empty(); | ||
| } | ||
| GenericRecord incomingRecord = bytesToAvro(recordBytes, schema); | ||
| /* | ||
| * Combining strategy here returns currentValue on disk if incoming record is older. | ||
| * The incoming record can be either a delete (sent as an upsert with _hoodie_is_deleted set to true) | ||
| * or an insert/update record. In any case, if it is older than the record in disk, the currentValue | ||
| * in disk is returned (to be rewritten with new commit time). | ||
| * | ||
| * NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path | ||
|
||
| * and need to be dealt with separately. | ||
| */ | ||
| Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue, props.get(ORDERING_FIELD_OPT_KEY), true); | ||
| Comparable incomingOrderingVal = (Comparable) getNestedFieldVal(incomingRecord, props.get(ORDERING_FIELD_OPT_KEY), false); | ||
|
||
|
|
||
| // Null check is needed here to support schema evolution. The record in storage may be from old schema where | ||
| // the new ordering column might not be present and hence returns null. | ||
| if (persistedOrderingVal != null && ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) > 0) { | ||
| return Option.of(currentValue); | ||
| } | ||
|
|
||
| /* | ||
| * We reached a point where the value is disk is older than the incoming record. | ||
| * Now check if the incoming record is a delete record. | ||
| */ | ||
| if (isDeleteRecord(incomingRecord)) { | ||
| return Option.empty(); | ||
| } else { | ||
| return Option.of(incomingRecord); | ||
| } | ||
| } | ||
| } | ||
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.
@vinothchandar @bhasudha : can you throw some light on why new config rather than reusing preCombine field value? I am working on putting up a new PR, so would like to know was this intentional or unintentional.
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.
That's because you don't know what the ordering field value is, when reading the data from storage right?
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.
ok, so for preCombine (combining two records within the same ingestion batch) we use one config and for determining ordering among one incoming record and another in storage or in other words, to combine an incoming record w/ one in storage, we use a diff config?
I see both these as one and the same. just that whether we resolve records within same batch or across batches is what differs.
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.
You are right, they are one and the same. If we can avoid the additional config, we should
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.
gotcha. here is my take.
Will introduce a config called "honorOrderingToCombineRecordsAcrossBatches" (we can work out a good naming). default value is false. Will send in orderingFieldKey (same as preCombineFieldKey) as an arg to OverwriteWithLatestAvroPayload.
Based on "honorOrderingToCombineRecordsAcrossBatches" value, combineAndGetUpdateValue() impl will decide to go with existing impl or new one (as we see in this patch)
I know we do have a gap here where in, if ordering field has changed over time, we can't do much here. But we can't afford to store the ordering field as a separate column in dataset either, as diff commits could theoretically have diff ordering field. But guess we can call it out that we may not support such evolution of ordering field.
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.
I prefer we don't do this. but separate the existing impl and new one into separate payload classes.
don't understand actually how we plan to implement obtaining the ordering field from the record in dfs. could you please clarify? @nsivabalan
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.
similar to how this patch already does. basically we lookup the preCombine field in existing generic record.
Reason I proposed to tweak existing class is.
Even if we want to go w/ 2 classes, Ideally I would want to name existing class as OverwriteWithIncomingPayload and introduce a new class called OverwriteWithLatestAvroPayload, bcoz, existing one overwrites w/ incoming and we are introduce a new class which will actually overwrite w/ latest payload.
But since users configure via class names, we can't go w/ this approach and hence my proposal to introduce a config and change existing class.