Add delete support to Hoodie (#85)
This commit is contained in:
committed by
prazanna
parent
41e08018fc
commit
348a48aa80
@@ -54,6 +54,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private RollingAvroLogAppender logAppender;
|
||||
private List<HoodieRecord<T>> records;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime,
|
||||
@@ -112,18 +113,24 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
try {
|
||||
IndexedRecord avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord, hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
||||
|
||||
if(avroRecord.isPresent()) {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord);
|
||||
recordsWritten++;
|
||||
return Optional.of(avroRecord);
|
||||
return avroRecord;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e);
|
||||
@@ -149,6 +156,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
logAppender.close();
|
||||
}
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
|
||||
Reference in New Issue
Block a user