[HUDI-2044] Integrate consumers with rocksDB and compression within External Spillable Map (#3318)
This commit is contained in:
@@ -51,14 +51,18 @@ import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.SchemaTestUtil;
|
||||
import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.exception.CorruptedLogFileException;
|
||||
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
@@ -73,6 +77,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -81,6 +86,7 @@ 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;
|
||||
import static org.junit.jupiter.params.provider.Arguments.arguments;
|
||||
|
||||
/**
|
||||
* Tests hoodie log format {@link HoodieLogFormat}.
|
||||
@@ -447,8 +453,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testBasicAppendAndScanMultipleFiles(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer =
|
||||
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
|
||||
@@ -487,6 +495,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
|
||||
List<IndexedRecord> scannedRecords = new ArrayList<>();
|
||||
@@ -594,8 +604,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderBasic(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
// Set a small threshold so that every block is a new version
|
||||
@@ -639,6 +651,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(200, scanner.getTotalLogRecords());
|
||||
Set<String> readKeys = new HashSet<>(200);
|
||||
@@ -652,8 +666,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithRollbackTombstone(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
// Set a small threshold so that every block is a new version
|
||||
@@ -713,6 +729,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches");
|
||||
Set<String> readKeys = new HashSet<>(200);
|
||||
@@ -725,8 +743,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
assertEquals(originalKeys, readKeys, "CompositeAvroLogReader should return 200 records from 2 versions");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvroLogRecordReaderWithFailedPartialBlock()
|
||||
@ParameterizedTest
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
// Set a small threshold so that every block is a new version
|
||||
@@ -796,6 +816,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records");
|
||||
Set<String> readKeys = new HashSet<>(200);
|
||||
@@ -809,8 +831,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
// Set a small threshold so that every block is a new version
|
||||
@@ -870,6 +894,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
|
||||
assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records");
|
||||
@@ -914,14 +940,18 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
|
||||
assertEquals(200, readKeys.size(), "Stream collect should return all 200 records after rollback of delete");
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithFailedRollbacks(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
|
||||
// Write a Data block and Delete block with same InstantTime (written in same batch)
|
||||
@@ -991,6 +1021,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(0, scanner.getTotalLogRecords(), "We would have scanned 0 records because of rollback");
|
||||
|
||||
@@ -1001,8 +1033,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithInsertDeleteAndRollback(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
|
||||
// Write a Data block and Delete block with same InstantTime (written in same batch)
|
||||
@@ -1055,14 +1089,18 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records");
|
||||
FileCreateUtils.deleteDeltaCommit(basePath, "100", fs);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithInvalidRollback(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
// Set a small threshold so that every block is a new version
|
||||
@@ -1102,6 +1140,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records");
|
||||
final List<String> readKeys = new ArrayList<>(100);
|
||||
@@ -1110,8 +1150,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
|
||||
// Write a 3 Data blocs with same InstantTime (written in same batch)
|
||||
@@ -1168,13 +1210,17 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records");
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(boolean readBlocksLazily)
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
|
||||
// Write a 3 Data blocs with same InstantTime (written in same batch)
|
||||
@@ -1270,6 +1316,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records");
|
||||
FileCreateUtils.deleteDeltaCommit(basePath, "100", fs);
|
||||
@@ -1289,7 +1337,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
*
|
||||
*/
|
||||
private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1, int numRecordsInLog2,
|
||||
boolean readBlocksLazily) {
|
||||
ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily) {
|
||||
try {
|
||||
// Write one Data block with same InstantTime (written in same batch)
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
@@ -1340,6 +1390,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(bufferSize)
|
||||
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
|
||||
.withDiskMapType(diskMapType)
|
||||
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
|
||||
.build();
|
||||
|
||||
assertEquals(Math.max(numRecordsInLog1, numRecordsInLog2), scanner.getNumMergedRecordsInLog(),
|
||||
@@ -1351,33 +1403,42 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt(boolean readBlocksLazily) {
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily) {
|
||||
/*
|
||||
* FIRST_ATTEMPT_FAILED:
|
||||
* Original task from the stage attempt failed, but subsequent stage retry succeeded.
|
||||
*/
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(77, 100, readBlocksLazily);
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(77, 100,
|
||||
diskMapType, isCompressionEnabled, readBlocksLazily);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt(boolean readBlocksLazily) {
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily) {
|
||||
/*
|
||||
* SECOND_ATTEMPT_FAILED:
|
||||
* Original task from stage attempt succeeded, but subsequent retry attempt failed.
|
||||
*/
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(100, 66, readBlocksLazily);
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(100, 66,
|
||||
diskMapType, isCompressionEnabled, readBlocksLazily);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testAvroLogRecordReaderTasksSucceededInBothStageAttempts(boolean readBlocksLazily) {
|
||||
@MethodSource("testArguments")
|
||||
public void testAvroLogRecordReaderTasksSucceededInBothStageAttempts(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
boolean isCompressionEnabled,
|
||||
boolean readBlocksLazily) {
|
||||
/*
|
||||
* BOTH_ATTEMPTS_SUCCEEDED:
|
||||
* Original task from the stage attempt and duplicate task from the stage retry succeeded.
|
||||
*/
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(100, 100, readBlocksLazily);
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(100, 100,
|
||||
diskMapType, isCompressionEnabled, readBlocksLazily);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -1618,4 +1679,18 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
throw new RuntimeException("Unknown data block type " + dataBlockType);
|
||||
}
|
||||
}
|
||||
|
||||
private static Stream<Arguments> testArguments() {
|
||||
// Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily
|
||||
return Stream.of(
|
||||
arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, true),
|
||||
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, true),
|
||||
arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, true),
|
||||
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, true)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table.view;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
|
||||
/**
|
||||
@@ -29,6 +30,7 @@ public class TestSpillableMapBasedFileSystemView extends TestHoodieTableFileSyst
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline, FileSystemViewStorageConfig.newBuilder()
|
||||
// pure disk base View
|
||||
.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).withMaxMemoryForView(0L).build());
|
||||
.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).withMaxMemoryForView(0L).build(),
|
||||
HoodieCommonConfig.newBuilder().build());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table.view;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
|
||||
@@ -29,6 +30,7 @@ public class TestSpillableMapBasedIncrementalFSViewSync extends TestIncrementalF
|
||||
@Override
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withMaxMemoryForView(0L).withIncrementalTimelineSync(true).build());
|
||||
FileSystemViewStorageConfig.newBuilder().withMaxMemoryForView(0L).withIncrementalTimelineSync(true).build(),
|
||||
HoodieCommonConfig.newBuilder().build());
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user