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

@@ -17,7 +17,6 @@
package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -180,21 +179,29 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
while (statusIterator.hasNext()) {
WriteStatus writeStatus = statusIterator.next();
List<Put> puts = new ArrayList<>();
List<Delete> deletes = new ArrayList<>();
try {
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(rec.getKey())) {
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
HoodieRecordLocation loc = rec.getNewLocation();
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
Bytes.toBytes(loc.getCommitTime()));
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
Bytes.toBytes(loc.getFileId()));
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
Bytes.toBytes(rec.getPartitionPath()));
puts.add(put);
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
if(loc.isPresent()) {
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
Bytes.toBytes(loc.get().getCommitTime()));
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
Bytes.toBytes(loc.get().getFileId()));
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
Bytes.toBytes(rec.getPartitionPath()));
puts.add(put);
} else {
//Delete existing index for a deleted record
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
deletes.add(delete);
}
}
}
hTable.put(puts);
hTable.delete(deletes);
hTable.flushCommits();
} catch (Exception e) {
Exception we = new Exception("Error updating index for " + writeStatus, e);

View File

@@ -41,9 +41,8 @@ import java.util.concurrent.ConcurrentMap;
/**
* Hoodie Index implementation backed by an in-memory Hash map.
*
* <p>
* ONLY USE FOR LOCAL TESTING
*
*/
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@@ -56,7 +55,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> hoodieTable) {
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> hoodieTable) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
@@ -67,7 +66,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
@Override
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();
@@ -82,7 +81,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) {
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@@ -94,7 +93,14 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
public WriteStatus call(WriteStatus writeStatus) {
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(record.getKey())) {
recordLocationMap.put(record.getKey(), record.getNewLocation());
HoodieKey key = record.getKey();
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
if (newLocation.isPresent()) {
recordLocationMap.put(key, newLocation.get());
} else {
//Delete existing index for a deleted record
recordLocationMap.remove(key);
}
}
}
return writeStatus;

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) {

View File

@@ -256,6 +256,85 @@ public class TestHoodieClient implements Serializable {
readClient.readCommit(newCommitTime).count(),
readClient.readSince("001").count());
}
@Test
public void testDeletes() throws Exception {
HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/**
* Write 1 (inserts and deletes)
* Write actual 200 insert records and ignore 100 delete records
*/
String newCommitTime = "001";
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(newCommitTime, 200);
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletes(newCommitTime, 100);
List<HoodieRecord> records = new ArrayList(fewRecordsForInsert);
records.addAll(fewRecordsForDelete);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
// verify that there is a commit
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
assertEquals("Expecting a single commit.", readClient.listCommitsSince("000").size(), 1);
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), fewRecordsForInsert.size());
// Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect();
checkTaggedRecords(taggedRecords, "001");
/**
* Write 2 (deletes+writes)
*/
newCommitTime = "004";
fewRecordsForDelete = records.subList(0,50);
List<HoodieRecord> fewRecordsForUpdate = records.subList(50,100);
records = dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete);
records.addAll(fewRecordsForUpdate);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
// verify there are now 2 commits
readClient = new HoodieReadClient(jsc, basePath, sqlContext);
assertEquals("Expecting two commits.", readClient.listCommitsSince("000").size(), 2);
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
// Check the entire dataset has 150 records(200-50) still
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
for (int i=0; i < fullPartitionPaths.length; i++) {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
}
assertEquals("Must contain 150 records", readClient.read(fullPartitionPaths).count(), 150);
// Check that the incremental consumption from time 000
assertEquals("Incremental consumption from latest commit, should give 50 updated records",
readClient.readCommit(newCommitTime).count(),
50);
assertEquals("Incremental consumption from time 001, should give 50 updated records",
50,
readClient.readSince("001").count());
assertEquals("Incremental consumption from time 000, should give 150",
150,
readClient.readSince("000").count());
}
@Test
public void testInsertAndCleanByVersions() throws Exception {
int maxVersions = 2; // keep upto 2 versions for each file

View File

@@ -34,10 +34,7 @@ import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import java.util.*;
/**
* Class to be used in tests to keep generating test inserts and updates against a corpus.
@@ -100,6 +97,26 @@ public class HoodieTestDataGenerator {
return inserts;
}
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts);
}
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords) throws IOException {
List<HoodieRecord> deletes = new ArrayList<>();
for (HoodieRecord existingRecord: existingRecords) {
HoodieRecord record = generateDeleteRecord(existingRecord);
deletes.add(record);
}
return deletes;
}
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
HoodieKey key = existingRecord.getKey();
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), null, true);
return new HoodieRecord(key, payload);
}
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();

View File

@@ -27,6 +27,7 @@ import org.apache.commons.io.IOUtils;
import java.io.*;
import java.util.Map;
import java.util.Optional;
import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
@@ -41,12 +42,22 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
private String rowKey;
private byte[] jsonDataCompressed;
private int dataSize;
private boolean isDeleted;
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath, String schemaStr) throws IOException {
this.jsonDataCompressed = compressData(jsonData);
this.dataSize = jsonData.length();
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
String schemaStr, Boolean isDeleted) throws IOException {
if(jsonData.isPresent()) {
this.jsonDataCompressed = compressData(jsonData.get());
this.dataSize = jsonData.get().length();
}
this.rowKey = rowKey;
this.partitionPath = partitionPath;
this.isDeleted = isDeleted;
}
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
String schemaStr)throws IOException {
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
}
public TestRawTripPayload(String jsonData) throws IOException {
@@ -55,6 +66,7 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
this.rowKey = jsonRecordMap.get("_row_key").toString();
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
this.isDeleted = false;
}
public String getPartitionPath() {
@@ -66,20 +78,24 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
return another;
}
@Override public IndexedRecord combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
@Override public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
return this.getInsertValue(schema);
}
@Override public IndexedRecord getInsertValue(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return jsonConverter.convert(getJsonData());
@Override public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if(isDeleted){
return Optional.empty();
} else {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
}
public String getRowKey() {
return rowKey;
}
public String getJsonData() throws IOException {
private String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
}

View File

@@ -421,7 +421,7 @@ public class TestHoodieBloomIndex {
int seqId = 1;
String commitTime = FSUtils.getCommitTime(filename);
for (HoodieRecord record : records) {
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema);
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
writer.write(avroRecord);