[HUDI-2029] Implement compression for DiskBasedMap in Spillable Map (#3128)
This commit is contained in:
@@ -38,6 +38,8 @@ import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
@@ -66,27 +68,33 @@ public class TestBitCaskDiskMap extends HoodieCommonTestHarness {
|
||||
initPath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleInsert() throws IOException, URISyntaxException {
|
||||
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath);
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testSimpleInsert(boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath, isCompressionEnabled);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
|
||||
Map<String, IndexedRecord> originalRecords = iRecords.stream()
|
||||
.collect(Collectors.toMap(k -> ((GenericRecord) k).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(), v -> v));
|
||||
|
||||
// make sure records have spilled to disk
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
List<HoodieRecord> oRecords = new ArrayList<>();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
oRecords.add(rec);
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
IndexedRecord originalRecord = originalRecords.get(rec.getRecordKey());
|
||||
HoodieAvroPayload payload = (HoodieAvroPayload) rec.getData();
|
||||
Option<IndexedRecord> value = payload.getInsertValue(HoodieAvroUtils.addMetadataFields(getSimpleSchema()));
|
||||
assertEquals(originalRecord, value.get());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
|
||||
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath);
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testSimpleInsertWithoutHoodieMetadata(boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath, isCompressionEnabled);
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
|
||||
Set<String> recordKeys = new HashSet<>();
|
||||
// insert generated records into the map
|
||||
@@ -105,11 +113,12 @@ public class TestBitCaskDiskMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleUpsert() throws IOException, URISyntaxException {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testSimpleUpsert(boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
|
||||
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath);
|
||||
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath, isCompressionEnabled);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
|
||||
// perform some inserts
|
||||
@@ -187,9 +196,10 @@ public class TestBitCaskDiskMap extends HoodieCommonTestHarness {
|
||||
assertTrue(payloadSize > 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPutAll() throws IOException, URISyntaxException {
|
||||
BitCaskDiskMap<String, HoodieRecord> records = new BitCaskDiskMap<>(basePath);
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testPutAll(boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
BitCaskDiskMap<String, HoodieRecord> records = new BitCaskDiskMap<>(basePath, isCompressionEnabled);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
Map<String, HoodieRecord> recordMap = new HashMap<>();
|
||||
iRecords.forEach(r -> {
|
||||
|
||||
@@ -39,7 +39,8 @@ import org.junit.jupiter.api.MethodOrderer.Alphanumeric;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestMethodOrder;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
@@ -48,11 +49,13 @@ import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
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 external spillable map {@link ExternalSpillableMap}.
|
||||
@@ -69,13 +72,13 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(ExternalSpillableMap.DiskMapType.class)
|
||||
public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
|
||||
@MethodSource("testArguments")
|
||||
public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); // 16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
@@ -102,14 +105,13 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(ExternalSpillableMap.DiskMapType.class)
|
||||
public void testSimpleUpsert(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
|
||||
|
||||
@MethodSource("testArguments")
|
||||
public void testSimpleUpsert(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); // 16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
@@ -140,15 +142,15 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(ExternalSpillableMap.DiskMapType.class)
|
||||
public void testAllMapOperations(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
|
||||
@MethodSource("testArguments")
|
||||
public void testAllMapOperations(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); // 16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
// insert a bunch of records so that values spill to disk too
|
||||
@@ -198,13 +200,13 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(ExternalSpillableMap.DiskMapType.class)
|
||||
public void simpleTestWithException(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
|
||||
@MethodSource("testArguments")
|
||||
public void simpleTestWithException(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records = new ExternalSpillableMap<>(16L,
|
||||
failureOutputPath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); // 16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
@@ -218,14 +220,14 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(ExternalSpillableMap.DiskMapType.class)
|
||||
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
|
||||
@MethodSource("testArguments")
|
||||
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); // 16B
|
||||
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
// Ensure we spill to disk
|
||||
@@ -271,14 +273,14 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(ExternalSpillableMap.DiskMapType.class)
|
||||
public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
|
||||
@MethodSource("testArguments")
|
||||
public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
|
||||
new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); // 16B
|
||||
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
// Ensure we spill to disk
|
||||
@@ -338,4 +340,13 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
// TODO : come up with a performance eval test for spillableMap
|
||||
@Test
|
||||
public void testLargeInsertUpsert() {}
|
||||
}
|
||||
|
||||
private static Stream<Arguments> testArguments() {
|
||||
return Stream.of(
|
||||
arguments(ExternalSpillableMap.DiskMapType.BITCASK, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.UNKNOWN, false),
|
||||
arguments(ExternalSpillableMap.DiskMapType.BITCASK, true)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user