1
0

Fixing Rollback for compaction/commit operation, added check for null commit

- Fallback to old way of rollback by listing all partitions
	- Added null check to ensure only partitions which are to be rolledback are considered
	- Added location (committime) to workload stat
	- Added checks in CompactedScanner to guard against task retries
	- Introduce new logic for rollback (bounded by instant_time and target_instant time)
        - Reversed logfiles order
This commit is contained in:
Nishith Agarwal
2017-12-14 21:34:54 -08:00
committed by vinoth chandar
parent be0b1f3e57
commit 2116815261
10 changed files with 348 additions and 162 deletions

View File

@@ -16,9 +16,6 @@
package com.uber.hoodie.common.table.log;
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;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
@@ -32,6 +29,14 @@ import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
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;
@@ -43,13 +48,9 @@ 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;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
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
@@ -76,8 +77,8 @@ public class HoodieCompactedLogRecordScanner implements
private HoodieTableMetaClient hoodieTableMetaClient;
// Merge strategy to use when combining records from log
private String payloadClassFQN;
// Store only the last log blocks (needed to implement rollback)
Deque<HoodieLogBlock> lastBlocks = new ArrayDeque<>();
// Store the last instant log blocks (needed to implement rollback)
Deque<HoodieLogBlock> currentInstantLogBlocks = new ArrayDeque<>();
public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema, String latestInstantTime) {
@@ -100,8 +101,8 @@ public class HoodieCompactedLogRecordScanner implements
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,
if (r.getBlockType() != CORRUPT_BLOCK &&
!HoodieTimeline.compareTimestamps(r.getLogMetadata().get(INSTANT_TIME), this.latestInstantTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
//hit a block with instant time greater than should be processed, stop processing further
break;
@@ -109,22 +110,33 @@ public class HoodieCompactedLogRecordScanner implements
switch (r.getBlockType()) {
case AVRO_DATA_BLOCK:
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, lastBlocks);
// store the last block
lastBlocks.push(r);
// Consider the following scenario
// (Time 0, C1, Task T1) -> Running
// (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1
// (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2)
// (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2)
// Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same.
// Say, commit C1 eventually failed and a rollback is triggered.
// Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file,
// but in reality we need to rollback (B1 & B2)
// The following code ensures the same rollback block (R1) is used to rollback both B1 & B2
if(isNewInstantBlock(r)) {
// If this is a avro data block, then merge the last block records into the main result
merge(records, currentInstantLogBlocks);
}
// store the current block
currentInstantLogBlocks.push(r);
break;
case DELETE_BLOCK:
log.info("Reading a delete block from file " + logFile.getPath());
String lastBlockInstantTime = lastBlocks.peek().getLogMetadata().get(INSTANT_TIME);
if (!lastBlockInstantTime.equals(blockInstantTime)) {
if (isNewInstantBlock(r)) {
// 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);
merge(records, currentInstantLogBlocks);
}
// store deletes so can be rolled back
lastBlocks.push(r);
currentInstantLogBlocks.push(r);
break;
case COMMAND_BLOCK:
log.info("Reading a command block from file " + logFile.getPath());
@@ -137,28 +149,46 @@ public class HoodieCompactedLogRecordScanner implements
// 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) {
// the rollback operation itself and ensures the same rollback block (R1) is used to rollback
// both B1 & B2 with same instant_time
int numBlocksRolledBack = 0;
while(!currentInstantLogBlocks.isEmpty()) {
HoodieLogBlock lastBlock = currentInstantLogBlocks.peek();
// 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());
if (lastBlock.getBlockType() == CORRUPT_BLOCK) {
log.info(
"Rolling back the last corrupted log block read in " + logFile.getPath());
currentInstantLogBlocks.pop();
numBlocksRolledBack++;
}
// rollback last data block or delete block
else if (lastBlock.getBlockType() != CORRUPT_BLOCK &&
targetInstantForCommandBlock
.contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
log.info("Rolling back the last log block read in " + logFile.getPath());
currentInstantLogBlocks.pop();
numBlocksRolledBack++;
}
// invalid or extra rollback block
else if(!targetInstantForCommandBlock
.contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME))) {
log.warn("Invalid or extra rollback command block in " + logFile.getPath());
break;
}
// this should not happen ideally
else {
log.warn("Unable to apply rollback command block in " + logFile.getPath());
}
}
log.info("Number of applied rollback blocks " + numBlocksRolledBack);
break;
}
break;
case CORRUPT_BLOCK:
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
lastBlocks.push(r);
currentInstantLogBlocks.push(r);
break;
}
}
@@ -167,15 +197,26 @@ public class HoodieCompactedLogRecordScanner implements
throw new HoodieIOException("IOException when reading log file " + logFile);
}
// merge the last read block when all the blocks are done reading
if (!lastBlocks.isEmpty()) {
if (!currentInstantLogBlocks.isEmpty()) {
log.info("Merging the final data blocks in " + logFile.getPath());
merge(records, lastBlocks);
merge(records, currentInstantLogBlocks);
}
}
this.logRecords = Collections.unmodifiableCollection(records.values());
this.totalRecordsToUpdate = records.size();
}
/**
* Checks if the current logblock belongs to a later instant
* @param logBlock
* @return
*/
private boolean isNewInstantBlock(HoodieLogBlock logBlock) {
return currentInstantLogBlocks.size() > 0 && currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK
&& !logBlock.getLogMetadata().get(INSTANT_TIME)
.contentEquals(currentInstantLogBlocks.peek().getLogMetadata().get(INSTANT_TIME));
}
/**
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge
* with the application specific payload if the same key was found before Sufficient to just merge
@@ -218,15 +259,18 @@ public class HoodieCompactedLogRecordScanner implements
private void merge(Map<String, HoodieRecord<? extends HoodieRecordPayload>> records,
Deque<HoodieLogBlock> lastBlocks) {
while (!lastBlocks.isEmpty()) {
HoodieLogBlock lastBlock = lastBlocks.pop();
// poll the element at the bottom of the stack since that's the order it was inserted
HoodieLogBlock lastBlock = lastBlocks.pollLast();
switch (lastBlock.getBlockType()) {
case AVRO_DATA_BLOCK:
merge(records, loadRecordsFromBlock((HoodieAvroDataBlock) lastBlock));
break;
case DELETE_BLOCK:
// TODO : If delete is the only block written and/or records are present in parquet file
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove);
break;
case CORRUPT_BLOCK:
log.warn("Found a corrupt block which was not rolled back");
break;
}
}

View File

@@ -520,23 +520,25 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
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
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
// Rollback the last write
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
// Write 3
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102");
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
@@ -552,8 +554,8 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("We only read 200 records, but only 200 of them are valid", 200,
schema, "102");
assertEquals("We read 200 records from 2 write batches", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -583,12 +585,13 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
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();
// Write 2
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
@@ -605,6 +608,8 @@ public class HoodieLogFormatTest {
outputStream.close();
// Rollback the last write
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
@@ -613,6 +618,7 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(commandBlock);
// Write 3
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103");
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
@@ -629,7 +635,7 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
schema, "103");
assertEquals("We would read 200 records", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -660,12 +666,12 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
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
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
@@ -682,6 +688,7 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "102");
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
metadata);
writer = writer.appendBlock(deleteBlock);
@@ -693,7 +700,7 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
schema, "102");
assertEquals("We still would read 200 records", 200,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(200);
@@ -706,12 +713,14 @@ public class HoodieLogFormatTest {
readKeys);
// Rollback the last block
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "103");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "102");
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "100");
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101");
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200,
readKeys.size());
@@ -756,7 +765,7 @@ public class HoodieLogFormatTest {
metadata);
writer = writer.appendBlock(deleteBlock);
// Attemp 1 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
// Attempt 1 : Write rollback block for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
try {
@@ -766,8 +775,7 @@ public class HoodieLogFormatTest {
} 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);
// Attempt 2 : Write another rollback blocks for a failed write
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
@@ -778,12 +786,13 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("We would read 100 records", 100,
// all data must be rolled back before merge
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
final List<String> readKeys = new ArrayList<>();
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());
assertEquals("Stream collect should return all 0 records", 0, readKeys.size());
}
@Test
@@ -850,12 +859,12 @@ public class HoodieLogFormatTest {
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)
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
@@ -873,4 +882,140 @@ public class HoodieLogFormatTest {
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());
}
@Test
public void testAvroLogRecordReaderWithInsertsDeleteAndRollback()
throws IOException, URISyntaxException, InterruptedException {
// Write a 3 Data blocs 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);
List<IndexedRecord> copyOfRecords1 = records1.stream().map(record ->
HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema))
.collect(Collectors.toList());
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 = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = copyOfRecords1.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 1 rollback block for a failed write
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
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, basePath,
allLogFiles, schema, "101");
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@Test
public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback()
throws IOException, URISyntaxException, InterruptedException {
// Write a 3 Data blocs 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);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
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.flush();
outputStream.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
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.flush();
outputStream.close();
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
writer = writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
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.flush();
outputStream.close();
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1 rollback block for a failed write
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "101");
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, basePath,
allLogFiles, schema, "101");
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
}