1
0

Enabling hard deletes for MergeOnRead table type

This commit is contained in:
Nishith Agarwal
2018-12-21 14:51:44 -08:00
committed by vinoth chandar
parent 345aaa31aa
commit 110df7190b
10 changed files with 103 additions and 39 deletions

View File

@@ -36,7 +36,11 @@ public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload>
public HoodieAvroPayload(Optional<GenericRecord> record) {
try {
this.recordBytes = HoodieAvroUtils.avroToBytes(record.get());
if (record.isPresent()) {
this.recordBytes = HoodieAvroUtils.avroToBytes(record.get());
} else {
this.recordBytes = new byte[0];
}
} catch (IOException io) {
throw new HoodieIOException("Cannot convert record to bytes", io);
}
@@ -55,6 +59,9 @@ public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload>
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (recordBytes.length == 0) {
return Optional.empty();
}
Optional<GenericRecord> record = Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema));
}

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.common.table.log;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
@@ -63,7 +64,7 @@ public abstract class AbstractHoodieLogRecordScanner {
private static final Logger log = LogManager.getLogger(AbstractHoodieLogRecordScanner.class);
// Reader schema for the records
private final Schema readerSchema;
protected final Schema readerSchema;
// Latest valid instant time
// Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark.
private final String latestInstantTime;
@@ -291,7 +292,7 @@ public abstract class AbstractHoodieLogRecordScanner {
*
* @param key Deleted record key
*/
protected abstract void processNextDeletedKey(String key);
protected abstract void processNextDeletedKey(HoodieKey key);
/**
* Process the set of log blocks belonging to the last instant which is read fully.

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.HoodieTimer;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
@@ -102,10 +103,11 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
}
@Override
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) {
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws IOException {
String key = hoodieRecord.getRecordKey();
if (records.containsKey(key)) {
// Merge and store the merged record
// Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be
// done when a delete (empty payload) is encountered before or after an insert/update.
HoodieRecordPayload combinedValue = records.get(key).getData().preCombine(hoodieRecord.getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
} else {
@@ -115,10 +117,9 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
}
@Override
protected void processNextDeletedKey(String key) {
// TODO : If delete is the only block written and/or records are present in parquet file
// TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry
records.remove(key);
protected void processNextDeletedKey(HoodieKey hoodieKey) {
records.put(hoodieKey.getRecordKey(), SpillableMapUtils.generateEmptyPayload(hoodieKey.getRecordKey(),
hoodieKey.getPartitionPath(), getPayloadClassFQN()));
}
public long getTotalTimeTakenToReadAndMergeBlocks() {

View File

@@ -18,6 +18,7 @@
package com.uber.hoodie.common.table.log;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.util.List;
@@ -43,7 +44,7 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordScann
}
@Override
protected void processNextDeletedKey(String key) {
protected void processNextDeletedKey(HoodieKey key) {
throw new IllegalStateException("Not expected to see delete records in this log-scan mode. Check Job Config");
}

View File

@@ -16,16 +16,16 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import com.uber.hoodie.common.util.StringUtils;
import com.uber.hoodie.common.util.SerializationUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
@@ -36,9 +36,9 @@ import org.apache.hadoop.fs.FSDataInputStream;
*/
public class HoodieDeleteBlock extends HoodieLogBlock {
private String[] keysToDelete;
private HoodieKey[] keysToDelete;
public HoodieDeleteBlock(String[] keysToDelete,
public HoodieDeleteBlock(HoodieKey[] keysToDelete,
Map<HeaderMetadataType, String> header) {
this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>());
this.keysToDelete = keysToDelete;
@@ -64,15 +64,14 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ",")
.getBytes(Charset.forName("utf-8"));
byte[] bytesToWrite = SerializationUtils.serialize(getKeysToDelete());
output.writeInt(HoodieLogBlock.version);
output.writeInt(bytesToWrite.length);
output.write(bytesToWrite);
return baos.toByteArray();
}
public String[] getKeysToDelete() {
public HoodieKey[] getKeysToDelete() {
try {
if (keysToDelete == null) {
if (!getContent().isPresent() && readBlockLazily) {
@@ -86,7 +85,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
int dataLength = dis.readInt();
byte[] data = new byte[dataLength];
dis.readFully(data);
this.keysToDelete = new String(data).split(",");
this.keysToDelete = SerializationUtils.deserialize(data);
deflate();
}
return keysToDelete;

View File

@@ -117,4 +117,15 @@ public class SpillableMapUtils {
.loadPayload(payloadClazz, new Object[]{Optional.of(rec)}, Optional.class));
return (R) hoodieRecord;
}
/**
* Utility method to convert bytes to HoodieRecord using schema and payload class
*/
public static <R> R generateEmptyPayload(String recKey, String partitionPath, String payloadClazz) {
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(recKey, partitionPath),
ReflectionUtils
.loadPayload(payloadClazz, new Object[]{Optional.empty()}, Optional.class));
return (R) hoodieRecord;
}
}