Skip to content
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

[HUDI-6961] Fixing DefaultHoodieRecordPayload to honor deletion based on meta field as well as custome delete marker #10150

Merged
merged 5 commits into from
Nov 22, 2023
Merged
Show file tree
Hide file tree
Changes from 4 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 @@ -23,6 +23,7 @@
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;

import org.apache.avro.Schema;
Expand All @@ -33,6 +34,7 @@
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicBoolean;

/**
* {@link HoodieRecordPayload} impl that honors ordering field in both preCombine and combineAndGetUpdateValue.
Expand All @@ -44,6 +46,8 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
public static final String DELETE_KEY = "hoodie.payload.delete.field";
public static final String DELETE_MARKER = "hoodie.payload.delete.marker";
private Option<Object> eventTime = Option.empty();
private AtomicBoolean isDeleteComputed = new AtomicBoolean(false);
private boolean isDefaultRecordPayloadDeleted = false;

public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal) {
super(record, orderingVal);
Expand Down Expand Up @@ -72,10 +76,13 @@ public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue
*/
eventTime = updateEventTime(incomingRecord, properties);

if (!isDeleteComputed.getAndSet(true)) {
isDefaultRecordPayloadDeleted = isDeleteRecord(incomingRecord, properties);
}
/*
* Now check if the incoming record is a delete record.
*/
return isDeleteRecord(incomingRecord, properties) ? Option.empty() : Option.of(incomingRecord);
return isDefaultRecordPayloadDeleted ? Option.empty() : Option.of(incomingRecord);
}

@Override
Expand All @@ -86,7 +93,25 @@ 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);
if (!isDeleteComputed.getAndSet(true)) {
isDefaultRecordPayloadDeleted = isDeleteRecord(incomingRecord, properties);
}
return isDefaultRecordPayloadDeleted ? Option.empty() : Option.of(incomingRecord);
}

public boolean isDeleted(Schema schema, Properties props) {
if (recordBytes.length == 0) {
return true;
}
try {
if (!isDeleteComputed.getAndSet(true)) {
GenericRecord incomingRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema);
isDefaultRecordPayloadDeleted = isDeleteRecord(incomingRecord, props);
}
return isDefaultRecordPayloadDeleted;
} catch (IOException e) {
throw new HoodieIOException("Deserializing bytes to avro failed ", e);
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,8 @@ public void testDeletedRecord(String key) throws IOException {

DefaultHoodieRecordPayload payload1 = new DefaultHoodieRecordPayload(record1, 1);
DefaultHoodieRecordPayload payload2 = new DefaultHoodieRecordPayload(delRecord1, 2);
assertFalse(payload1.isDeleted(schema, props));
assertTrue(payload2.isDeleted(schema, props));
assertEquals(payload1.preCombine(payload2, props), payload2);
assertEquals(payload2.preCombine(payload1, props), payload2);

Expand Down Expand Up @@ -145,9 +147,13 @@ public void testDeleteKey() throws IOException {
DefaultHoodieRecordPayload deletePayload = new DefaultHoodieRecordPayload(delRecord, 2);
DefaultHoodieRecordPayload defaultDeletePayload = new DefaultHoodieRecordPayload(defaultDeleteRecord, 2);

assertFalse(payload.isDeleted(schema, props));
assertTrue(deletePayload.isDeleted(schema, props));
assertFalse(defaultDeletePayload.isDeleted(schema, props)); // if custom marker is present, should honor that irrespective of hoodie_is_deleted

assertEquals(record, payload.getInsertValue(schema, props).get());
assertEquals(defaultDeleteRecord, defaultDeletePayload.getInsertValue(schema, props).get());
assertFalse(deletePayload.getInsertValue(schema, props).isPresent());
assertTrue(defaultDeletePayload.getInsertValue(schema, props).isPresent()); // if custom marker is present, should honor that irrespective of hoodie_is_deleted

assertEquals(delRecord, payload.combineAndGetUpdateValue(delRecord, schema, props).get());
assertEquals(defaultDeleteRecord, payload.combineAndGetUpdateValue(defaultDeleteRecord, schema, props).get());
Expand Down
Loading