-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-6961] Fix deletes with custom delete field in DefaultHoodieRecordPayload #9892
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
d2fd9c4
721f4ef
8a3d6d6
f8cee72
c580c03
db35ccd
e73788c
187aa0b
f2a25c4
088db01
d6eab69
355a341
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 |
|---|---|---|
|
|
@@ -45,12 +45,12 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload { | |
| public static final String DELETE_MARKER = "hoodie.payload.delete.marker"; | ||
| private Option<Object> eventTime = Option.empty(); | ||
|
|
||
| public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal) { | ||
| super(record, orderingVal); | ||
| public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal, Properties props) { | ||
| super(record, orderingVal, props); | ||
| } | ||
|
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. The source of the props seems a chaos, I already saw several ways how it was produced:
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. Yeah, this should be cleaned up separately. Regardless, the payload-related configs should always be included so there's no correctness issue. And we're not storing any props with the record so record processing is consistent. |
||
|
|
||
| public DefaultHoodieRecordPayload(Option<GenericRecord> record) { | ||
| this(record.isPresent() ? record.get() : null, 0); // natural order | ||
| public DefaultHoodieRecordPayload(Option<GenericRecord> record, Properties props) { | ||
| this(record.isPresent() ? record.get() : null, 0, props); // natural order | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -75,7 +75,7 @@ public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue | |
| /* | ||
| * Now check if the incoming record is a delete record. | ||
| */ | ||
| return isDeleteRecord(incomingRecord, properties) ? Option.empty() : Option.of(incomingRecord); | ||
| return isDeleted(schema, properties) ? Option.empty() : Option.of(incomingRecord); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -86,30 +86,26 @@ public Option<IndexedRecord> getInsertValue(Schema schema, Properties properties | |
| GenericRecord incomingRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); | ||
| eventTime = updateEventTime(incomingRecord, properties); | ||
|
|
||
| return isDeleteRecord(incomingRecord, properties) ? Option.empty() : Option.of(incomingRecord); | ||
| return isDeleted(schema, properties) ? Option.empty() : Option.of(incomingRecord); | ||
yihua marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| /** | ||
| * @param genericRecord instance of {@link GenericRecord} of interest. | ||
| * @param properties payload related properties | ||
| * @returns {@code true} if record represents a delete record. {@code false} otherwise. | ||
| */ | ||
| protected boolean isDeleteRecord(GenericRecord genericRecord, Properties properties) { | ||
| final String deleteKey = properties.getProperty(DELETE_KEY); | ||
| @Override | ||
| protected boolean isDeleteRecord(GenericRecord record, Properties props) { | ||
| final String deleteKey = props.getProperty(DELETE_KEY); | ||
| if (StringUtils.isNullOrEmpty(deleteKey)) { | ||
| return isDeleteRecord(genericRecord); | ||
| return super.isDeleteRecord(record, props); | ||
|
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. Is this line the actual fix, I didn't see the props got used by the super method, so do we still need to pass around all the props here?
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. The implementation of this method is kept the same. However, before my fix, this method After the fix, the method
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. But the We can address the props issue is separate task maybe.
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. The
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.
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'm talking about the maste code, in master,
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. The intention is that |
||
| } | ||
|
|
||
| ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(properties.getProperty(DELETE_MARKER)), | ||
| ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(props.getProperty(DELETE_MARKER)), | ||
| () -> DELETE_MARKER + " should be configured with " + DELETE_KEY); | ||
| // Modify to be compatible with new version Avro. | ||
| // The new version Avro throws for GenericRecord.get if the field name | ||
| // does not exist in the schema. | ||
| if (genericRecord.getSchema().getField(deleteKey) == null) { | ||
| if (record.getSchema().getField(deleteKey) == null) { | ||
| return false; | ||
| } | ||
| Object deleteMarker = genericRecord.get(deleteKey); | ||
| return deleteMarker != null && properties.getProperty(DELETE_MARKER).equals(deleteMarker.toString()); | ||
| Object deleteMarker = record.get(deleteKey); | ||
| return deleteMarker != null && props.getProperty(DELETE_MARKER).equals(deleteMarker.toString()); | ||
| } | ||
|
|
||
| private static Option<Object> updateEventTime(GenericRecord record, Properties properties) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,9 @@ | |
| import org.apache.avro.generic.IndexedRecord; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Properties; | ||
|
|
||
| import static org.apache.hudi.common.util.ConfigUtils.EMPTY_PROPS; | ||
|
|
||
| /** | ||
| * This is a payload to wrap a existing Hoodie Avro Record. Useful to create a HoodieRecord over existing GenericRecords | ||
|
|
@@ -39,11 +42,19 @@ public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload> | |
| private final Comparable<?> orderingVal; | ||
|
|
||
| public HoodieAvroPayload(GenericRecord record, Comparable<?> orderingVal) { | ||
| this(record, orderingVal, EMPTY_PROPS); | ||
|
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. shouldn't we mark these as deprecated ?
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. Based on my understanding this is only used internally and props are not used, so I don't mark it as deprecated. |
||
| } | ||
|
|
||
| public HoodieAvroPayload(Option<GenericRecord> record) { | ||
| this(record, EMPTY_PROPS); | ||
| } | ||
|
|
||
| public HoodieAvroPayload(GenericRecord record, Comparable<?> orderingVal, Properties props) { | ||
| this.recordBytes = record == null ? new byte[0] : HoodieAvroUtils.avroToBytes(record); | ||
| this.orderingVal = orderingVal; | ||
| } | ||
|
|
||
| public HoodieAvroPayload(Option<GenericRecord> record) { | ||
| public HoodieAvroPayload(Option<GenericRecord> record, Properties props) { | ||
| this.recordBytes = record.isPresent() ? HoodieAvroUtils.avroToBytes(record.get()) : new byte[0]; | ||
| this.orderingVal = 0; | ||
| } | ||
|
|
||

Uh oh!
There was an error while loading. Please reload this page.