capture record metadata before deflating for record counting
This commit is contained in:
committed by
vinoth chandar
parent
f2980052cd
commit
c98ee057fc
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
@@ -66,7 +67,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
* Load the new incoming records in a map, and extract the old file path.
|
||||
*/
|
||||
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
WriteStatus writeStatus = new WriteStatus();
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
@@ -129,6 +130,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
|
||||
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
if(indexedRecord.isPresent()) {
|
||||
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
|
||||
@@ -139,11 +141,11 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
}
|
||||
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord);
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record "+ hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user