1
0

Add delete support to Hoodie (#85)

This commit is contained in:
siddharthagunda
2017-03-04 01:33:49 -08:00
committed by prazanna
parent 41e08018fc
commit 348a48aa80
16 changed files with 241 additions and 68 deletions

View File

@@ -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);

View File

@@ -34,6 +34,7 @@ import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.Optional;
import java.util.UUID;
public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
@@ -42,7 +43,8 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
private final WriteStatus status;
private final HoodieStorageWriter<IndexedRecord> storageWriter;
private final Path path;
private int recordsWritten = 0;
private long recordsWritten = 0;
private long recordsDeleted = 0;
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable, String partitionPath) {
@@ -82,13 +84,19 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
*/
public void write(HoodieRecord record) {
try {
IndexedRecord avroRecord = record.getData().getInsertValue(schema);
storageWriter.writeAvroWithMetadata(avroRecord, record);
status.markSuccess(record);
// update the new location of record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
if(avroRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
// update the new location of record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
recordsWritten++;
} else {
recordsDeleted++;
}
record.deflate();
recordsWritten++;
status.markSuccess(record);
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
@@ -111,6 +119,7 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
HoodieWriteStat stat = new HoodieWriteStat();
stat.setNumWrites(recordsWritten);
stat.setNumDeletes(recordsDeleted);
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
stat.setFileId(status.getFileId());
stat.setFullPath(path.toString());

View File

@@ -37,6 +37,7 @@ import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Optional;
@SuppressWarnings("Duplicates")
public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
@@ -48,6 +49,7 @@ public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIO
private Path newFilePath;
private Path oldFilePath;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
private String fileId;
@@ -118,13 +120,18 @@ public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIO
}
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, IndexedRecord indexedRecord) {
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
try {
storageWriter.writeAvroWithMetadata(indexedRecord, hoodieRecord);
if(indexedRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
recordsWritten++;
updatedRecordsWritten++;
} else {
recordsDeleted++;
}
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord);
recordsWritten ++;
updatedRecordsWritten ++;
return true;
} catch (Exception e) {
logger.error("Error writing record "+ hoodieRecord, e);
@@ -142,8 +149,8 @@ public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIO
boolean copyOldRecord = true;
if (keyToNewRecords.containsKey(key)) {
try {
IndexedRecord avroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
if (writeUpdateRecord(hoodieRecord, avroRecord)) {
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
/* ONLY WHEN
* 1) we have an update for this key AND
* 2) We are able to successfully write the the combined new value
@@ -194,8 +201,10 @@ public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIO
if (storageWriter != null) {
storageWriter.close();
}
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {