1
0

1) Separated rollback as a table operation 2) Implement rollback for MOR

This commit is contained in:
Nishith Agarwal
2017-08-17 17:31:55 -07:00
committed by vinoth chandar
parent e1fe3ab937
commit c7d63a7622
28 changed files with 1200 additions and 260 deletions

View File

@@ -16,10 +16,9 @@
package com.uber.hoodie.common;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import org.apache.hadoop.fs.FileStatus;
import java.io.File;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
@@ -34,12 +33,19 @@ public class HoodieRollbackStat implements Serializable {
private final List<String> successDeleteFiles;
// Files that could not be deleted
private final List<String> failedDeleteFiles;
// Count of HoodieLogFile to commandBlocks written for a particular rollback
private final Map<FileStatus, Long> commandBlocksCount;
public HoodieRollbackStat(String partitionPath, List<String> successDeleteFiles,
List<String> failedDeleteFiles) {
List<String> failedDeleteFiles, Map<FileStatus, Long> commandBlocksCount) {
this.partitionPath = partitionPath;
this.successDeleteFiles = successDeleteFiles;
this.failedDeleteFiles = failedDeleteFiles;
this.commandBlocksCount = commandBlocksCount;
}
public Map<FileStatus, Long> getCommandBlocksCount() {
return commandBlocksCount;
}
public String getPartitionPath() {
@@ -61,6 +67,7 @@ public class HoodieRollbackStat implements Serializable {
public static class Builder {
private List<String> successDeleteFiles;
private List<String> failedDeleteFiles;
private Map<FileStatus, Long> commandBlocksCount;
private String partitionPath;
public Builder withDeletedFileResults(Map<FileStatus, Boolean> deletedFiles) {
@@ -72,13 +79,18 @@ public class HoodieRollbackStat implements Serializable {
return this;
}
public Builder withRollbackBlockAppendResults(Map<FileStatus, Long> commandBlocksCount) {
this.commandBlocksCount = commandBlocksCount;
return this;
}
public Builder withPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
return this;
}
public HoodieRollbackStat build() {
return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles);
return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles, commandBlocksCount);
}
}
}

View File

@@ -21,20 +21,12 @@ import com.uber.hoodie.common.model.HoodieAvroPayload;
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.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -43,6 +35,21 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Deque;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME;
/**
* Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged
* list of records which will be used as a lookup table when merging the base columnar file
@@ -63,69 +70,99 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
private AtomicLong totalLogRecords = new AtomicLong(0);
// Total final list of compacted/merged records
private long totalRecordsToUpdate;
// Latest valid instant time
private String latestInstantTime;
public HoodieCompactedLogRecordScanner(FileSystem fs, List<String> logFilePaths,
Schema readerSchema) {
Schema readerSchema, String latestInstantTime) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
// Store only the last log blocks (needed to implement rollback)
Deque<HoodieLogBlock> lastBlocks = new ArrayDeque<>();
// Store merged records for all versions for this log file
Map<String, HoodieRecord<HoodieAvroPayload>> records = Maps.newHashMap();
// iterate over the paths
logFilePaths.stream().map(s -> new HoodieLogFile(new Path(s))).forEach(s -> {
log.info("Scanning log file " + s.getPath());
Iterator<String> logFilePathsItr = logFilePaths.iterator();
while(logFilePathsItr.hasNext()) {
HoodieLogFile logFile = new HoodieLogFile(new Path(logFilePathsItr.next()));
log.info("Scanning log file " + logFile.getPath());
totalLogFiles.incrementAndGet();
try {
// Use the HoodieLogFormatReader to iterate through the blocks in the log file
HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, s, readerSchema);
// Store the records loaded from the last data block (needed to implement rollback)
Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock = Maps.newHashMap();
reader.forEachRemaining(r -> {
HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, logFile, readerSchema, true);
while(reader.hasNext()) {
HoodieLogBlock r = reader.next();
String blockInstantTime = r.getLogMetadata().get(INSTANT_TIME);
if(!HoodieTimeline.compareTimestamps(blockInstantTime, this.latestInstantTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
//hit a block with instant time greater than should be processed, stop processing further
break;
}
switch (r.getBlockType()) {
case AVRO_DATA_BLOCK:
log.info("Reading a data block from file " + s.getPath());
log.info("Reading a data block from file " + logFile.getPath());
// If this is a avro data block, then merge the last block records into the main result
merge(records, recordsFromLastBlock);
// Load the merged records into recordsFromLastBlock
HoodieAvroDataBlock dataBlock = (HoodieAvroDataBlock) r;
loadRecordsFromBlock(dataBlock, recordsFromLastBlock);
merge(records, lastBlocks);
// store the last block
lastBlocks.push(r);
break;
case DELETE_BLOCK:
log.info("Reading a delete block from file " + s.getPath());
// This is a delete block, so lets merge any records from previous data block
merge(records, recordsFromLastBlock);
// Delete the keys listed as to be deleted
HoodieDeleteBlock deleteBlock = (HoodieDeleteBlock) r;
Arrays.stream(deleteBlock.getKeysToDelete()).forEach(records::remove);
log.info("Reading a delete block from file " + logFile.getPath());
String lastBlockInstantTime = lastBlocks.peek().getLogMetadata().get(INSTANT_TIME);
if(!lastBlockInstantTime.equals(blockInstantTime)) {
// Block with the keys listed as to be deleted, data and delete blocks written in different batches
// so it is safe to merge
// This is a delete block, so lets merge any records from previous data block
merge(records, lastBlocks);
}
// store deletes so can be rolled back
lastBlocks.push(r);
break;
case COMMAND_BLOCK:
log.info("Reading a command block from file " + s.getPath());
log.info("Reading a command block from file " + logFile.getPath());
// This is a command block - take appropriate action based on the command
HoodieCommandBlock commandBlock = (HoodieCommandBlock) r;
if (commandBlock.getType() == HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK) {
log.info("Rolling back the last data block read in " + s.getPath());
// rollback the last read data block
recordsFromLastBlock.clear();
String targetInstantForCommandBlock = r.getLogMetadata().get(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME);
switch (commandBlock.getType()) { // there can be different types of command blocks
case ROLLBACK_PREVIOUS_BLOCK:
// Rollback the last read log block
// Get commit time from last record block, compare with targetCommitTime, rollback only if equal,
// this is required in scenarios of invalid/extra rollback blocks written due to failures during
// the rollback operation itself
HoodieLogBlock lastBlock = lastBlocks.peek();
if (lastBlock != null && lastBlock.getBlockType() != CORRUPT_BLOCK &&
targetInstantForCommandBlock.contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
log.info("Rolling back the last log block read in " + logFile.getPath());
lastBlocks.pop();
} else if(lastBlock != null && lastBlock.getBlockType() == CORRUPT_BLOCK) {
// handle corrupt blocks separately since they may not have metadata
log.info("Rolling back the last corrupted log block read in " + logFile.getPath());
lastBlocks.pop();
}
else {
log.warn("Invalid or extra rollback command block in " + logFile.getPath());
}
break;
}
break;
case CORRUPT_BLOCK:
log.info("Found a corrupt block in " + s.getPath());
log.info("Found a corrupt block in " + logFile.getPath());
// If there is a corrupt block - we will assume that this was the next data block
// so merge the last block records (TODO - handle when the corrupted block was a tombstone written partially?)
merge(records, recordsFromLastBlock);
recordsFromLastBlock.clear();
lastBlocks.push(r);
break;
}
});
// merge the last read block when all the blocks are done reading
if (!recordsFromLastBlock.isEmpty()) {
log.info("Merging the final data block in " + s.getPath());
merge(records, recordsFromLastBlock);
}
} catch (IOException e) {
throw new HoodieIOException("IOException when reading log file " + s);
throw new HoodieIOException("IOException when reading log file " + logFile);
}
});
// merge the last read block when all the blocks are done reading
if(!lastBlocks.isEmpty()) {
log.info("Merging the final data blocks in " + logFile.getPath());
merge(records, lastBlocks);
}
}
this.logRecords = Collections.unmodifiableCollection(records.values());
this.totalRecordsToUpdate = records.size();
}
@@ -135,17 +172,14 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
* and merge with the HoodieAvroPayload if the same key was found before
*
* @param dataBlock
* @param recordsFromLastBlock
*/
private void loadRecordsFromBlock(
HoodieAvroDataBlock dataBlock,
Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock) {
recordsFromLastBlock.clear();
private Map<String, HoodieRecord<HoodieAvroPayload>> loadRecordsFromBlock(HoodieAvroDataBlock dataBlock) {
Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock = Maps.newHashMap();
List<IndexedRecord> recs = dataBlock.getRecords();
totalLogRecords.addAndGet(recs.size());
recs.forEach(rec -> {
String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)
.toString();
.toString();
String partitionPath =
((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
.toString();
@@ -155,15 +189,39 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
if (recordsFromLastBlock.containsKey(key)) {
// Merge and store the merged record
HoodieAvroPayload combinedValue = recordsFromLastBlock.get(key).getData()
.preCombine(hoodieRecord.getData());
.preCombine(hoodieRecord.getData());
recordsFromLastBlock
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
} else {
// Put the record as is
recordsFromLastBlock.put(key, hoodieRecord);
}
});
return recordsFromLastBlock;
}
/**
* Merge the last seen log blocks with the accumulated records
*
* @param records
* @param lastBlocks
*/
private void merge(Map<String, HoodieRecord<HoodieAvroPayload>> records,
Deque<HoodieLogBlock> lastBlocks) {
while (!lastBlocks.isEmpty()) {
HoodieLogBlock lastBlock = lastBlocks.pop();
switch (lastBlock.getBlockType()) {
case AVRO_DATA_BLOCK:
merge(records, loadRecordsFromBlock((HoodieAvroDataBlock) lastBlock));
break;
case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove);
break;
case CORRUPT_BLOCK:
break;
}
}
}
/**
@@ -178,9 +236,9 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
if (records.containsKey(key)) {
// Merge and store the merged record
HoodieAvroPayload combinedValue = records.get(key).getData()
.preCombine(hoodieRecord.getData());
.preCombine(hoodieRecord.getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
combinedValue));
} else {
// Put the record as is
records.put(key, hoodieRecord);

View File

@@ -187,8 +187,8 @@ public interface HoodieLogFormat {
return new WriterBuilder();
}
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean readMetadata)
throws IOException {
return new HoodieLogFormatReader(fs, logFile, readerSchema);
return new HoodieLogFormatReader(fs, logFile, readerSchema, readMetadata);
}
}

View File

@@ -51,15 +51,17 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private static final byte[] magicBuffer = new byte[4];
private final Schema readerSchema;
private HoodieLogBlock nextBlock = null;
private boolean readMetadata = true;
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize) throws IOException {
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, boolean readMetadata) throws IOException {
this.inputStream = fs.open(logFile.getPath(), bufferSize);
this.logFile = logFile;
this.readerSchema = readerSchema;
this.readMetadata = readMetadata;
}
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE);
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean readMetadata) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readMetadata);
}
@Override
@@ -93,11 +95,11 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
switch (blockType) {
// based on type read the block
case AVRO_DATA_BLOCK:
return HoodieAvroDataBlock.fromBytes(content, readerSchema);
return HoodieAvroDataBlock.fromBytes(content, readerSchema, readMetadata);
case DELETE_BLOCK:
return HoodieDeleteBlock.fromBytes(content);
return HoodieDeleteBlock.fromBytes(content, readMetadata);
case COMMAND_BLOCK:
return HoodieCommandBlock.fromBytes(content);
return HoodieCommandBlock.fromBytes(content, readMetadata);
default:
throw new HoodieNotSupportedException("Unsupported Block " + blockType);
}
@@ -113,7 +115,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
int corruptedBlockSize = (int) (nextBlockOffset - currentPos);
byte[] content = new byte[corruptedBlockSize];
inputStream.readFully(content, 0, corruptedBlockSize);
return HoodieCorruptBlock.fromBytes(content);
return HoodieCorruptBlock.fromBytes(content, corruptedBlockSize, true);
}
private boolean isBlockCorrupt(int blocksize) throws IOException {

View File

@@ -18,13 +18,6 @@ package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.util.HoodieAvroUtils;
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.util.ArrayList;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
@@ -34,6 +27,15 @@ import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
/**
* DataBlock contains a list of records serialized using Avro.
* The Datablock contains
@@ -43,16 +45,21 @@ import org.apache.avro.io.EncoderFactory;
* 4. Size of a record
* 5. Actual avro serialized content of the record
*/
public class HoodieAvroDataBlock implements HoodieLogBlock {
public class HoodieAvroDataBlock extends HoodieLogBlock {
private List<IndexedRecord> records;
private Schema schema;
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) {
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema, Map<LogMetadataType, String> metadata) {
super(metadata);
this.records = records;
this.schema = schema;
}
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) {
this(records, schema, null);
}
public List<IndexedRecord> getRecords() {
return records;
}
@@ -63,19 +70,25 @@ public class HoodieAvroDataBlock implements HoodieLogBlock {
@Override
public byte[] getBytes() throws IOException {
GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
// 1. Compress and Write schema out
// 1. Write out metadata
if(super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
// 2. Compress and Write schema out
byte[] schemaContent = HoodieAvroUtils.compress(schema.toString());
output.writeInt(schemaContent.length);
output.write(schemaContent);
// 2. Write total number of records
// 3. Write total number of records
output.writeInt(records.size());
// 3. Write the records
// 4. Write the records
records.forEach(s -> {
ByteArrayOutputStream temp = new ByteArrayOutputStream();
Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null);
@@ -104,9 +117,15 @@ public class HoodieAvroDataBlock implements HoodieLogBlock {
return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema) throws IOException {
// 1. Read the schema written out
public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
// 1. Read the metadata written out, if applicable
if(readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);
}
// 1. Read the schema written out
int schemaLength = dis.readInt();
byte[] compressedSchema = new byte[schemaLength];
dis.readFully(compressedSchema, 0, schemaLength);
@@ -133,6 +152,6 @@ public class HoodieAvroDataBlock implements HoodieLogBlock {
}
dis.close();
return new HoodieAvroDataBlock(records, readerSchema);
return new HoodieAvroDataBlock(records, readerSchema, metadata);
}
}

View File

@@ -16,25 +16,42 @@
package com.uber.hoodie.common.table.log.block;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Map;
/**
* Command block issues a specific command to the scanner
*/
public class HoodieCommandBlock implements HoodieLogBlock {
public class HoodieCommandBlock extends HoodieLogBlock {
private final HoodieCommandBlockTypeEnum type;
public enum HoodieCommandBlockTypeEnum {ROLLBACK_PREVIOUS_BLOCK}
public HoodieCommandBlock(HoodieCommandBlockTypeEnum type) {
public HoodieCommandBlock(HoodieCommandBlockTypeEnum type, Map<LogMetadataType, String> metadata) {
super(metadata);
this.type = type;
}
public HoodieCommandBlock(HoodieCommandBlockTypeEnum type) {
this(type, null);
}
@Override
public byte[] getBytes() throws IOException {
return ByteBuffer.allocate(4).putInt(type.ordinal()).array();
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
if(super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
output.writeInt(type.ordinal());
output.close();
return baos.toByteArray();
}
public HoodieCommandBlockTypeEnum getType() {
@@ -46,8 +63,13 @@ public class HoodieCommandBlock implements HoodieLogBlock {
return HoodieLogBlockType.COMMAND_BLOCK;
}
public static HoodieLogBlock fromBytes(byte[] content) {
int ordinal = ByteBuffer.wrap(content).getInt();
return new HoodieCommandBlock(HoodieCommandBlockTypeEnum.values()[ordinal]);
public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
if(readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);
}
int ordinal = dis.readInt();
return new HoodieCommandBlock(HoodieCommandBlockTypeEnum.values()[ordinal], metadata);
}
}

View File

@@ -16,23 +16,39 @@
package com.uber.hoodie.common.table.log.block;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Map;
/**
* Corrupt block is emitted whenever the scanner finds the length of the block written at the
* beginning does not match (did not find a EOF or a sync marker after the length)
*/
public class HoodieCorruptBlock implements HoodieLogBlock {
public class HoodieCorruptBlock extends HoodieLogBlock {
private final byte[] corruptedBytes;
private HoodieCorruptBlock(byte[] corruptedBytes) {
private HoodieCorruptBlock(byte[] corruptedBytes, Map<LogMetadataType, String> metadata) {
super(metadata);
this.corruptedBytes = corruptedBytes;
}
private HoodieCorruptBlock(byte[] corruptedBytes) {
this(corruptedBytes, null);
}
@Override
public byte[] getBytes() throws IOException {
return corruptedBytes;
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
if(super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
output.write(corruptedBytes);
return baos.toByteArray();
}
@Override
@@ -40,7 +56,25 @@ public class HoodieCorruptBlock implements HoodieLogBlock {
return HoodieLogBlockType.CORRUPT_BLOCK;
}
public static HoodieLogBlock fromBytes(byte[] content) {
return new HoodieCorruptBlock(content);
public byte[] getCorruptedBytes() {
return corruptedBytes;
}
public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
int bytesRemaining = blockSize;
if(readMetadata) {
try { //attempt to read metadata
metadata = HoodieLogBlock.getLogMetadata(dis);
bytesRemaining = blockSize - HoodieLogBlock.getLogMetadataBytes(metadata).length;
} catch(IOException e) {
// unable to read metadata, possibly corrupted
metadata = null;
}
}
byte [] corruptedBytes = new byte[bytesRemaining];
dis.readFully(corruptedBytes);
return new HoodieCorruptBlock(corruptedBytes, metadata);
}
}

View File

@@ -16,24 +16,43 @@
package com.uber.hoodie.common.table.log.block;
import org.apache.commons.lang3.StringUtils;
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 org.apache.commons.lang3.StringUtils;
import java.util.Map;
/**
* Delete block contains a list of keys to be deleted from scanning the blocks so far
*/
public class HoodieDeleteBlock implements HoodieLogBlock {
public class HoodieDeleteBlock extends HoodieLogBlock {
private final String[] keysToDelete;
public HoodieDeleteBlock(String[] keysToDelete) {
public HoodieDeleteBlock(String[] keysToDelete, Map<LogMetadataType, String> metadata) {
super(metadata);
this.keysToDelete = keysToDelete;
}
public HoodieDeleteBlock(String[] keysToDelete) {
this(keysToDelete, null);
}
@Override
public byte[] getBytes() throws IOException {
return StringUtils.join(keysToDelete, ',').getBytes(Charset.forName("utf-8"));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
if(super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
byte [] bytesToWrite = StringUtils.join(keysToDelete, ',').getBytes(Charset.forName("utf-8"));
output.writeInt(bytesToWrite.length);
output.write(bytesToWrite);
return baos.toByteArray();
}
public String[] getKeysToDelete() {
@@ -45,7 +64,15 @@ public class HoodieDeleteBlock implements HoodieLogBlock {
return HoodieLogBlockType.DELETE_BLOCK;
}
public static HoodieLogBlock fromBytes(byte[] content) {
return new HoodieDeleteBlock(new String(content).split(","));
public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
if(readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);
}
int dataLength = dis.readInt();
byte [] data = new byte[dataLength];
dis.readFully(data);
return new HoodieDeleteBlock(new String(data).split(","), metadata);
}
}

View File

@@ -16,23 +16,105 @@
package com.uber.hoodie.common.table.log.block;
import com.google.common.collect.Maps;
import com.uber.hoodie.exception.HoodieException;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.Map;
/**
* Abstract interface defining a block in HoodieLogFile
* Abstract class defining a block in HoodieLogFile
*/
public interface HoodieLogBlock {
byte[] getBytes() throws IOException;
HoodieLogBlockType getBlockType();
public abstract class HoodieLogBlock {
public byte[] getBytes() throws IOException {
throw new HoodieException("No implementation was provided");
}
public HoodieLogBlockType getBlockType() {
throw new HoodieException("No implementation was provided");
}
//log metadata for each log block
private Map<LogMetadataType, String> logMetadata;
/**
* Type of the log block
* WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
*/
enum HoodieLogBlockType {
public enum HoodieLogBlockType {
COMMAND_BLOCK,
DELETE_BLOCK,
CORRUPT_BLOCK,
AVRO_DATA_BLOCK
}
/**
* Metadata abstraction for a HoodieLogBlock
* WARNING : This enum is serialized as the ordinal. Only add new enums at the end.
*/
public enum LogMetadataType {
INSTANT_TIME,
TARGET_INSTANT_TIME;
}
public HoodieLogBlock(Map<LogMetadataType, String> logMetadata) {
this.logMetadata = logMetadata;
}
public Map<LogMetadataType, String> getLogMetadata() {
return logMetadata;
}
/**
* Convert log metadata to bytes
* 1. Write size of metadata
* 2. Write enum ordinal
* 3. Write actual bytes
* @param metadata
* @return
* @throws IOException
*/
public static byte [] getLogMetadataBytes(Map<LogMetadataType, String> metadata) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
output.writeInt(metadata.size());
for(Map.Entry<LogMetadataType, String> entry : metadata.entrySet()) {
output.writeInt(entry.getKey().ordinal());
byte [] bytes = entry.getValue().getBytes();
output.writeInt(bytes.length);
output.write(bytes);
}
return baos.toByteArray();
}
/**
* Convert bytes to LogMetadata, follow the same order as
* {@link HoodieLogBlock#getLogMetadataBytes}
* @param dis
* @return
* @throws IOException
*/
public static Map<LogMetadataType, String> getLogMetadata(DataInputStream dis) throws IOException {
Map<LogMetadataType, String> metadata = Maps.newHashMap();
// 1. Read the metadata written out
int metadataCount = dis.readInt();
try {
while (metadataCount > 0) {
int metadataEntryIndex = dis.readInt();
int metadataEntrySize = dis.readInt();
byte[] metadataEntry = new byte[metadataEntrySize];
dis.readFully(metadataEntry, 0, metadataEntrySize);
metadata.put(LogMetadataType.values()[metadataEntryIndex], new String(metadataEntry));
metadataCount--;
}
return metadata;
} catch(EOFException eof) {
throw new IOException("Could not read metadata fields ", eof);
}
}
}

View File

@@ -256,7 +256,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
return readDataFromPath(detailPath);
}
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
Optional<byte[]> data) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
@@ -290,6 +289,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
public void saveToInflight(HoodieInstant instant, Optional<byte[]> content) {
createFileInMetaPath(instant.getFileName(), content);
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.common.model;
import com.google.common.collect.Lists;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.table.HoodieTableConfig;
@@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
@@ -249,6 +250,8 @@ public class HoodieTestUtils {
.overBaseCommit(location.getCommitTime())
.withFs(fs).build();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, location.getCommitTime());
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
@@ -260,7 +263,7 @@ public class HoodieTestUtils {
} catch (IOException e) {
return null;
}
}).collect(Collectors.toList()), schema));
}).collect(Collectors.toList()), schema, metadata));
logWriter.close();
} catch (Exception e) {
fail(e.toString());

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.common.table.log;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -24,16 +25,13 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -49,7 +47,12 @@ import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
@@ -107,8 +110,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
assertTrue("We just wrote a block - size should be > 0", size > 0);
@@ -124,8 +129,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
// Write out a block
writer = writer.appendBlock(dataBlock);
// Get the size of the block
@@ -138,7 +145,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
assertEquals("This should be a new log file and hence size should be 0", 0,
writer.getCurrentSize());
@@ -152,8 +159,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -163,7 +172,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
@@ -178,7 +187,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2);
@@ -202,8 +211,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
// do not close this writer - this simulates a data note appending to a log dying without closing the file
@@ -214,7 +225,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
@@ -232,12 +243,14 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("We wrote a block, we should be able to read it", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK,
@@ -257,8 +270,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -267,7 +282,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -277,11 +292,11 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
@@ -312,8 +327,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -324,20 +341,24 @@ public class HoodieLogFormatTest {
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.writeInt(1000);
// Write out some metadata
// TODO : test for failure to write metadata - NA ?
outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// First round of reads - we should be able to read the first block and then EOF
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should have corrupted block next", reader.hasNext());
HoodieLogBlock block = reader.next();
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
assertEquals("", "something-random", new String(block.getBytes()));
HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block;
assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes()));
assertFalse("There should be no more block left", reader.hasNext());
// Simulate another failure back to back
@@ -346,7 +367,10 @@ public class HoodieLogFormatTest {
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.writeInt(1000);
// Write out some metadata
// TODO : test for failure to write metadata - NA ?
outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
outputStream.write("something-else-random".getBytes());
outputStream.flush();
outputStream.close();
@@ -356,12 +380,12 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema());
dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
// Second round of reads - we should be able to read the first and last block
reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should get the 1st corrupted block next", reader.hasNext());
@@ -370,7 +394,8 @@ public class HoodieLogFormatTest {
block = reader.next();
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
assertEquals("", "something-else-random", new String(block.getBytes()));
corruptBlock = (HoodieCorruptBlock) block;
assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes()));
assertTrue("We should get the last block next", reader.hasNext());
reader.next();
assertFalse("We should have no more blocks left", reader.hasNext());
@@ -388,12 +413,15 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -402,8 +430,7 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -428,22 +455,27 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
// Rollback the last write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
// Write 3
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3, schema);
dataBlock = new HoodieAvroDataBlock(records3, schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -452,9 +484,8 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
assertEquals("We still would read 300 records, but only 200 of them are valid", 300,
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We only read 200 records, since 200 of them are valid", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -479,7 +510,11 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -491,13 +526,16 @@ public class HoodieLogFormatTest {
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
// Write out some metadata
// TODO : test for failure to write metadata - NA ?
outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// Rollback the last write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
@@ -505,7 +543,7 @@ public class HoodieLogFormatTest {
// Write 3
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3, schema);
dataBlock = new HoodieAvroDataBlock(records3, schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -514,8 +552,7 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We would read 200 records", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -531,7 +568,7 @@ public class HoodieLogFormatTest {
}
@Test
public void testAvroLogRecordReaderWithDelete()
public void testAvroLogRecordReaderWithDeleteAndRollback()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
@@ -541,12 +578,16 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
records1.addAll(records2);
@@ -557,7 +598,8 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]));
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
writer = writer.appendBlock(deleteBlock);
List<String> allLogFiles = FSUtils
@@ -565,17 +607,170 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We still would read 200 records", 200,
scanner.getTotalLogRecords());
List<String> readKeys = new ArrayList<>(200);
final List<String> readKeys = new ArrayList<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 150, readKeys.size());
assertEquals("Stream collect should return all 150 records", 150, readKeys.size());
originalKeys.removeAll(deletedKeys);
Collections.sort(originalKeys);
Collections.sort(readKeys);
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys,
assertEquals("CompositeAvroLogReader should return 150 records from 2 versions", originalKeys,
readKeys);
// Rollback the last block
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size());
}
@Test
public void testAvroLogRecordReaderWithFailedRollbacks()
throws IOException, URISyntaxException, InterruptedException {
// Write a Data block and Delete block with same InstantTime (written in same batch)
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
writer = writer.appendBlock(deleteBlock);
// Attemp 1 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
try {
writer = writer.appendBlock(commandBlock);
// Say job failed, retry writing 2 rollback in the next rollback(..) attempt
throw new Exception("simulating failure");
} catch(Exception e) {
// it's okay
}
// Attempt 2 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
writer = writer.appendBlock(commandBlock);
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We would read 100 records", 100,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());
}
@Test
public void testAvroLogRecordReaderWithInsertDeleteAndRollback()
throws IOException, URISyntaxException, InterruptedException {
// Write a Data block and Delete block with same InstantTime (written in same batch)
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
writer = writer.appendBlock(deleteBlock);
// Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@Test
public void testAvroLogRecordReaderWithInvalidRollback() throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write invalid rollback for a failed write (possible for in-flight commits)
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We still would read 100 records", 100,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());
}
}