1
0

[HUDI-2900] Fix corrupt block end position (#4181)

* [HUDI-2900] Fix corrupt block end position

* add a test
This commit is contained in:
Ron
2021-12-06 20:38:39 +08:00
committed by GitHub
parent f0e46bf522
commit 84b531ae75
3 changed files with 60 additions and 3 deletions

View File

@@ -603,6 +603,63 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
reader.close();
}
@Test
public void testValidateCorruptBlockEndPosition() throws IOException, URISyntaxException, InterruptedException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to the 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);
// Write out a length that does not confirm with the content
outputStream.writeLong(474);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
outputStream.writeInt(HoodieLogFormat.CURRENT_VERSION);
// Write out a length that does not confirm with the content
outputStream.writeLong(400);
// Write out incomplete content
outputStream.write("something-random".getBytes());
// get corrupt block end position
long corruptBlockEndPos = outputStream.getPos();
outputStream.flush();
outputStream.close();
// Append a proper block again
writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 10);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
writer.appendBlock(dataBlock);
writer.close();
// Read data and corrupt block
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue(reader.hasNext(), "First block should be available");
reader.next();
assertTrue(reader.hasNext(), "We should have corrupted block next");
HoodieLogBlock block = reader.next();
assertEquals(HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType(), "The read block should be a corrupt block");
// validate the corrupt block end position correctly.
assertEquals(corruptBlockEndPos, block.getBlockContentLocation().get().getBlockEndPos());
assertTrue(reader.hasNext(), "Third block should be available");
reader.next();
assertFalse(reader.hasNext(), "There should be no more block left");
reader.close();
}
@ParameterizedTest
@MethodSource("testArguments")
public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMapType,