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

@@ -30,6 +30,7 @@ import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Optional;
import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
@@ -47,13 +48,13 @@ public class HoodieJsonPayload implements HoodieRecordPayload<HoodieJsonPayload>
return this;
}
@Override public IndexedRecord combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
@Override public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
return getInsertValue(schema);
}
@Override public IndexedRecord getInsertValue(Schema schema) throws IOException {
@Override public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return jsonConverter.convert(getJsonData());
return Optional.of(jsonConverter.convert(getJsonData()));
}
private String getJsonData() throws IOException {

View File

@@ -17,6 +17,9 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -28,9 +31,9 @@ import java.io.IOException;
* Useful to create a HoodieRecord over existing GenericRecords in a hoodie datasets (useful in compactions)
*/
public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload> {
private final GenericRecord record;
private final Optional<GenericRecord> record;
public HoodieAvroPayload(GenericRecord record) {
public HoodieAvroPayload(Optional<GenericRecord> record) {
this.record = record;
}
@@ -40,13 +43,13 @@ public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload>
}
@Override
public IndexedRecord combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public IndexedRecord getInsertValue(Schema schema) throws IOException {
return HoodieAvroUtils.rewriteRecord(record, schema);
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema));
}
}

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.common.model;
import com.google.common.base.Objects;
import java.io.Serializable;
import java.util.Optional;
/**
* A Single Record managed by Hoodie TODO - Make this generic
@@ -101,8 +102,8 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
return this;
}
public HoodieRecordLocation getNewLocation() {
return this.newLocation;
public Optional<HoodieRecordLocation> getNewLocation() {
return Optional.of(this.newLocation);
}
public boolean isCurrentLocationKnown() {

View File

@@ -21,6 +21,7 @@ import org.apache.avro.generic.IndexedRecord;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
/**
* Every Hoodie dataset has an implementation of the <code>HoodieRecordPayload</code>
@@ -44,14 +45,15 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri
*
* @param currentValue Current value in storage, to merge/combine this payload with
* @param schema Schema used for record
* @return new combined/merged value to be written back to storage
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this record.
*/
IndexedRecord combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException;
Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException;
/**
* Generates an avro record out of the given HoodieRecordPayload, to be written out to storage.
* Called when writing a new value for the given HoodieKey, wherein there is no existing record in
* storage to be combined against. (i.e insert)
* Return EMPTY to skip writing this record.
*/
IndexedRecord getInsertValue(Schema schema) throws IOException;
Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException;
}

View File

@@ -51,6 +51,11 @@ public class HoodieWriteStat implements Serializable {
*/
private long numWrites;
/**
* Total number of records deleted.
*/
private long numDeletes;
/**
* Total number of records actually changed. (0 for inserts)
*/
@@ -86,6 +91,10 @@ public class HoodieWriteStat implements Serializable {
this.numWrites = numWrites;
}
public void setNumDeletes(long numDeletes) {
this.numDeletes = numDeletes;
}
public void setNumUpdateWrites(long numUpdateWrites) {
this.numUpdateWrites = numUpdateWrites;
}
@@ -110,6 +119,10 @@ public class HoodieWriteStat implements Serializable {
return numWrites;
}
public long getNumDeletes() {
return numDeletes;
}
public long getNumUpdateWrites() {
return numUpdateWrites;
}
@@ -129,6 +142,7 @@ public class HoodieWriteStat implements Serializable {
.append("fullPath='" + fullPath + '\'')
.append(", prevCommit='" + prevCommit + '\'')
.append(", numWrites=" + numWrites)
.append(", numDeletes=" + numDeletes)
.append(", numUpdateWrites=" + numUpdateWrites)
.append(", numWriteBytes=" + totalWriteBytes)
.append('}')

View File

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
public class AvroUtils {
@@ -58,7 +59,7 @@ public class AvroUtils {
String partitionPath =
deltaRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
loadedRecords.add(new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(deltaRecord)));
new HoodieAvroPayload(Optional.of(deltaRecord))));
}
fileReader.close(); // also closes underlying FsInput
} catch (IOException e) {