1
0

[HUDI-1434] fix incorrect log file path in HoodieWriteStat (#2300)

* [HUDI-1434] fix incorrect log file path in HoodieWriteStat

* HoodieWriteHandle#close() returns a list of WriteStatus objs

* Handle rolled-over log files and return a WriteStatus per log file written

 - Combined data and delete block logging into a single call
 - Lazily initialize and manage write status based on returned AppendResult
 - Use FSUtils.getFileSize() to set final file size, consistent with other handles
 - Added tests around returned values in AppendResult
 - Added validation of the file sizes returned in write stat

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Gary Li
2020-12-31 06:22:15 +08:00
committed by GitHub
parent ef28763f08
commit 605b617cfa
29 changed files with 591 additions and 298 deletions

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.log.AppendResult;
import org.apache.hudi.common.table.log.HoodieLogFileReader;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
@@ -76,6 +77,7 @@ import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -119,7 +121,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
}
@Test
public void testEmptyLog() throws IOException, InterruptedException {
public void testEmptyLog() throws IOException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
@@ -138,18 +140,21 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
long pos = writer.getCurrentSize();
HoodieDataBlock dataBlock = getDataBlock(dataBlockType, records, header);
writer = writer.appendBlock(dataBlock);
AppendResult result = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
assertTrue(size > 0, "We just wrote a block - size should be > 0");
assertEquals(size, fs.getFileStatus(writer.getLogFile().getPath()).getLen(),
"Write should be auto-flushed. The size reported by FileStatus and the writer should match");
assertEquals(size, result.size());
assertEquals(writer.getLogFile(), result.logFile());
assertEquals(0, result.offset());
writer.close();
}
@ParameterizedTest
@EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" })
@Test
public void testRollover() throws IOException, InterruptedException, URISyntaxException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
@@ -160,23 +165,36 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
// Write out a block
writer = writer.appendBlock(dataBlock);
AppendResult firstAppend = writer.appendBlock(dataBlock);
// Get the size of the block
long size = writer.getCurrentSize();
writer.close();
assertEquals(0, firstAppend.offset());
assertEquals(size, firstAppend.size());
// Create a writer with the size threshold as the size we just wrote - so this has to roll
writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
AppendResult secondAppend = writer.appendBlock(dataBlock);
assertEquals(firstAppend.logFile(), secondAppend.logFile());
assertNotEquals(0, secondAppend.offset());
assertEquals(0, writer.getCurrentSize(), "This should be a new log file and hence size should be 0");
assertEquals(2, writer.getLogFile().getLogVersion(), "Version should be rolled to 2");
Path logFilePath = writer.getLogFile().getPath();
assertFalse(fs.exists(logFilePath), "Path (" + logFilePath + ") must not exist");
// Write one more block, which should not go to the new log file.
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header);
AppendResult rolloverAppend = writer.appendBlock(dataBlock);
assertNotEquals(secondAppend.logFile(), rolloverAppend.logFile());
assertEquals(0, rolloverAppend.offset());
writer.close();
}
@@ -203,17 +221,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
if (newLogFileFormat && logFileExists) {
// Assume there is an existing log-file with write token
builder1 = builder1.withLogVersion(1).withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
builder2 = builder2.withLogVersion(1).withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
builder1 = builder1.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
builder2 = builder2.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
} else if (newLogFileFormat) {
// First log file of the file-slice
builder1 = builder1.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION)
.withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
builder2 = builder2.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION)
.withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
} else {
builder1 = builder1.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
@@ -224,9 +238,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
Writer writer2 = builder2.build();
writer2 = writer2.appendBlock(dataBlock);
writer2.appendBlock(dataBlock);
HoodieLogFile logFile1 = writer.getLogFile();
HoodieLogFile logFile2 = writer2.getLogFile();
writer.close();
@@ -245,7 +259,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -255,7 +269,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue(size2 > size1, "We just wrote a new block - size2 should be > size1");
assertEquals(size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen(),
@@ -269,7 +283,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue(size3 > size2, "We just wrote a new block - size3 should be > size2");
assertEquals(size3, fs.getFileStatus(writer.getLogFile().getPath()).getLen(),
@@ -325,9 +339,11 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieDataBlock dataBlock = getDataBlock(records, header);
for (int i = 0; i < 2; i++) {
HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive").overBaseCommit("")
.withFs(localFs).build().appendBlock(dataBlock).close();
.withFs(localFs).build();
writer.appendBlock(dataBlock);
writer.close();
}
// ensure there are two log file versions, with same data.
@@ -335,8 +351,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertEquals(2, statuses.length);
}
@ParameterizedTest
@EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" })
@Test
public void testBasicWriteAndScan() throws IOException, URISyntaxException, InterruptedException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
@@ -349,7 +364,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
@@ -364,8 +379,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
reader.close();
}
@ParameterizedTest
@EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" })
@Test
public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
@@ -378,7 +392,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
writer =
@@ -389,7 +403,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Close and Open again and append 100 more records
@@ -401,7 +415,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
@@ -455,7 +469,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
allRecords.add(copyOfRecords1);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
}
writer.close();
@@ -495,7 +509,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
@@ -521,7 +535,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
records = SchemaTestUtil.generateTestRecords(0, 10);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// First round of reads - we should be able to read the first block and then EOF
@@ -559,7 +573,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Second round of reads - we should be able to read the first and last block
@@ -597,7 +611,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
@@ -605,7 +619,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
List<String> allLogFiles =
@@ -654,21 +668,21 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Write 2
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Rollback the last write
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101");
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
writer.appendBlock(commandBlock);
// Write 3
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102");
@@ -677,7 +691,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
List<String> allLogFiles =
@@ -724,7 +738,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Write 2
@@ -756,7 +770,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
writer = writer.appendBlock(commandBlock);
writer.appendBlock(commandBlock);
// Write 3
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103");
@@ -766,7 +780,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
List<String> allLogFiles =
@@ -814,7 +828,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Write 2
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
@@ -822,7 +836,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
copyOfRecords1.addAll(copyOfRecords2);
List<String> originalKeys =
@@ -837,7 +851,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102");
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
writer = writer.appendBlock(deleteBlock);
writer.appendBlock(deleteBlock);
List<String> allLogFiles =
FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
@@ -922,13 +936,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Delete 50 keys
// Delete 50 keys
@@ -938,14 +952,14 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.collect(Collectors.toList()).subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
writer = writer.appendBlock(deleteBlock);
writer.appendBlock(deleteBlock);
// Attempt 1 : Write rollback block for a failed write
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
try {
writer = writer.appendBlock(commandBlock);
writer.appendBlock(commandBlock);
// Say job failed, retry writing 2 rollback in the next rollback(..) attempt
throw new Exception("simulating failure");
} catch (Exception e) {
@@ -999,7 +1013,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Delete 50 keys
List<HoodieKey> deletedKeys = copyOfRecords1.stream()
@@ -1007,13 +1021,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString())))
.collect(Collectors.toList()).subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
writer = writer.appendBlock(deleteBlock);
writer.appendBlock(deleteBlock);
// Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
writer.appendBlock(commandBlock);
writer.appendBlock(commandBlock);
List<String> allLogFiles =
@@ -1051,7 +1065,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Write invalid rollback for a failed write (possible for in-flight commits)
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101");
@@ -1103,9 +1117,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Delete 50 keys
// Delete 50 keys
@@ -1114,7 +1128,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString())))
.collect(Collectors.toList()).subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
writer = writer.appendBlock(deleteBlock);
writer.appendBlock(deleteBlock);
// Write 1 rollback block for a failed write
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
@@ -1160,9 +1174,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to the end of the log (mimics a partially written commit)
@@ -1195,7 +1209,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to the end of the log (mimics a partially written commit)
@@ -1220,7 +1234,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
writer.appendBlock(commandBlock);
writer.close();
List<String> allLogFiles =
@@ -1272,7 +1286,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records.subList(0, numRecordsInLog1), header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// Get the size of the block
long size = writer.getCurrentSize();
writer.close();
@@ -1286,7 +1300,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header2.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header2.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock2 = getDataBlock(records2.subList(0, numRecordsInLog2), header2);
writer2 = writer2.appendBlock(dataBlock2);
writer2.appendBlock(dataBlock2);
// Get the size of the block
writer2.close();
@@ -1360,7 +1374,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
writer =
@@ -1370,7 +1384,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Close and Open again and append 100 more records
@@ -1381,7 +1395,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(),
@@ -1429,7 +1443,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
@@ -1455,7 +1469,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// First round of reads - we should be able to read the first block and then EOF
@@ -1488,7 +1502,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
writer =
@@ -1496,7 +1510,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
// Close and Open again and append 100 more records
@@ -1505,7 +1519,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.close();
HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(),

View File

@@ -110,7 +110,7 @@ public class TestHoodieLogFormatAppendFailure {
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive")
.overBaseCommit("").withFs(fs).build();
writer = writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
// get the current log file version to compare later
int logFileVersion = writer.getLogFile().getLogVersion();
Path logFilePath = writer.getLogFile().getPath();

View File

@@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Tests hoodie delta write stat {@link HoodieDeltaWriteStat}.
*/
public class TestHoodieDeltaWriteStat {
@Test
public void testBaseFileAndLogFiles() {
HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
String baseFile = "file1.parquet";
String logFile1 = ".log1.log";
String logFile2 = ".log2.log";
writeStat.setBaseFile(baseFile);
writeStat.addLogFiles(logFile1);
writeStat.addLogFiles(logFile2);
assertTrue(writeStat.getLogFiles().contains(logFile1));
assertTrue(writeStat.getLogFiles().contains(logFile2));
assertEquals(baseFile, writeStat.getBaseFile());
writeStat.setLogFiles(new ArrayList<>());
assertTrue(writeStat.getLogFiles().isEmpty());
}
}