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
@@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user