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

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