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:
committed by
vinoth chandar
parent
be0b1f3e57
commit
2116815261
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user