[HUDI-2398] Collect event time for inserts in DefaultHoodieRecordPayload (#3602)
This commit is contained in:
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
@@ -56,7 +55,7 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
|
||||
if (recordBytes.length == 0) {
|
||||
return Option.empty();
|
||||
}
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
|
||||
GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
|
||||
|
||||
// Null check is needed here to support schema evolution. The record in storage may be from old schema where
|
||||
@@ -68,17 +67,27 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
|
||||
/*
|
||||
* We reached a point where the value is disk is older than the incoming record.
|
||||
*/
|
||||
eventTime = Option.ofNullable(getNestedFieldVal(incomingRecord, hoodieConfig
|
||||
.getString(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), true));
|
||||
eventTime = updateEventTime(incomingRecord, properties);
|
||||
|
||||
/*
|
||||
* Now check if the incoming record is a delete record.
|
||||
*/
|
||||
if (isDeleteRecord(incomingRecord)) {
|
||||
return isDeleteRecord(incomingRecord) ? Option.empty() : Option.of(incomingRecord);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema, Properties properties) throws IOException {
|
||||
if (recordBytes.length == 0) {
|
||||
return Option.empty();
|
||||
} else {
|
||||
return Option.of(incomingRecord);
|
||||
}
|
||||
GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
|
||||
eventTime = updateEventTime(incomingRecord, properties);
|
||||
|
||||
return isDeleteRecord(incomingRecord) ? Option.empty() : Option.of(incomingRecord);
|
||||
}
|
||||
|
||||
private static Option<Object> updateEventTime(GenericRecord record, Properties properties) {
|
||||
return Option.ofNullable(getNestedFieldVal(record, properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), true));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Reference in New Issue
Block a user