1
0

[HUDI-2028] Implement RockDbBasedMap as an alternate to DiskBasedMap in ExternalSpillableMap (#3194)

Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
This commit is contained in:
rmahindra123
2021-07-05 23:03:41 -07:00
committed by GitHub
parent a0f598d371
commit a4dcbb5c5a
14 changed files with 620 additions and 60 deletions

View File

@@ -57,9 +57,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Tests dis based map {@link DiskBasedMap}.
* Tests dis based map {@link BitCaskDiskMap}.
*/
public class TestDiskBasedMap extends HoodieCommonTestHarness {
public class TestBitCaskDiskMap extends HoodieCommonTestHarness {
@BeforeEach
public void setup() {
@@ -68,7 +68,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
@Test
public void testSimpleInsert() throws IOException, URISyntaxException {
DiskBasedMap records = new DiskBasedMap<>(basePath);
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath);
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);
@@ -86,7 +86,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
@Test
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
DiskBasedMap records = new DiskBasedMap<>(basePath);
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath);
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
Set<String> recordKeys = new HashSet<>();
// insert generated records into the map
@@ -109,7 +109,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
public void testSimpleUpsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
DiskBasedMap records = new DiskBasedMap<>(basePath);
BitCaskDiskMap records = new BitCaskDiskMap<>(basePath);
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
// perform some inserts
@@ -189,7 +189,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
@Test
public void testPutAll() throws IOException, URISyntaxException {
DiskBasedMap<String, HoodieRecord> records = new DiskBasedMap<>(basePath);
BitCaskDiskMap<String, HoodieRecord> records = new BitCaskDiskMap<>(basePath);
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<String, HoodieRecord> recordMap = new HashMap<>();
iRecords.forEach(r -> {

View File

@@ -38,6 +38,8 @@ import org.junit.jupiter.api.BeforeEach;
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 java.io.IOException;
import java.io.UncheckedIOException;
@@ -45,6 +47,7 @@ import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -65,32 +68,48 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
failureOutputPath = basePath + "/test_fail";
}
@Test
public void simpleInsertTest() throws IOException, URISyntaxException {
@ParameterizedTest
@EnumSource(ExternalSpillableMap.DiskMapType.class)
public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType) 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)); // 16B
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
// Test iterator
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
List<HoodieRecord> oRecords = new ArrayList<>();
int cntSize = 0;
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
oRecords.add(rec);
cntSize++;
assert recordKeys.contains(rec.getRecordKey());
}
assertEquals(recordKeys.size(), cntSize);
// Test value stream
List<HoodieRecord<? extends HoodieRecordPayload>> values = records.valueStream().collect(Collectors.toList());
cntSize = 0;
for (HoodieRecord value : values) {
assert recordKeys.contains(value.getRecordKey());
cntSize++;
}
assertEquals(recordKeys.size(), cntSize);
}
@Test
public void testSimpleUpsert() throws IOException, URISyntaxException {
@ParameterizedTest
@EnumSource(ExternalSpillableMap.DiskMapType.class)
public void testSimpleUpsert(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
@@ -120,14 +139,16 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
});
}
@Test
public void testAllMapOperations() throws IOException, URISyntaxException {
@ParameterizedTest
@EnumSource(ExternalSpillableMap.DiskMapType.class)
public void testAllMapOperations(ExternalSpillableMap.DiskMapType diskMapType) 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)); // 16B
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
// insert a bunch of records so that values spill to disk too
@@ -176,12 +197,14 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
assertTrue(records.size() == 0);
}
@Test
public void simpleTestWithException() throws IOException, URISyntaxException {
@ParameterizedTest
@EnumSource(ExternalSpillableMap.DiskMapType.class)
public void simpleTestWithException(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records = new ExternalSpillableMap<>(16L,
failureOutputPath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B
failureOutputPath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
@@ -194,13 +217,15 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
});
}
@Test
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException {
@ParameterizedTest
@EnumSource(ExternalSpillableMap.DiskMapType.class)
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk
@@ -245,13 +270,15 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
}
@Test
public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException {
@ParameterizedTest
@EnumSource(ExternalSpillableMap.DiskMapType.class)
public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException {
Schema schema = SchemaTestUtil.getSimpleSchema();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B
new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(schema), diskMapType); // 16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk
@@ -311,4 +338,4 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
// TODO : come up with a performance eval test for spillableMap
@Test
public void testLargeInsertUpsert() {}
}
}

View File

@@ -38,7 +38,7 @@ import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* Tests RocksDB based map {@link RocksDBBasedMap}.
* Tests RocksDB based map {@link RocksDbDiskMap}.
*/
public class TestRocksDbBasedMap extends HoodieCommonTestHarness {
@@ -49,7 +49,7 @@ public class TestRocksDbBasedMap extends HoodieCommonTestHarness {
@Test
public void testSimple() throws IOException, URISyntaxException {
RocksDBBasedMap records = new RocksDBBasedMap(basePath);
RocksDbDiskMap records = new RocksDbDiskMap(basePath);
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);

View File

@@ -0,0 +1,204 @@
/*
* 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.util.collection;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.common.testutils.SpillableMapTestUtils;
import org.apache.hudi.common.util.Option;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URISyntaxException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
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.assertTrue;
/**
* Test the rocksDb based Map {@link RocksDbDiskMap}
* that is used by {@link ExternalSpillableMap}.
*/
public class TestRocksDbDiskMap extends HoodieCommonTestHarness {
@BeforeEach
public void setUp() {
initPath();
}
@Test
public void testSimpleInsertSequential() throws IOException, URISyntaxException {
RocksDbDiskMap<String, HoodieRecord<? extends HoodieRecordPayload>> rocksDBBasedMap = new RocksDbDiskMap<>(basePath);
List<String> recordKeys = setupMapWithRecords(rocksDBBasedMap, 100);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = rocksDBBasedMap.iterator();
int cntSize = 0;
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
cntSize++;
assert recordKeys.contains(rec.getRecordKey());
}
assertEquals(recordKeys.size(), cntSize);
// Test value stream
long currentTimeMs = System.currentTimeMillis();
List<HoodieRecord<? extends HoodieRecordPayload>> values =
rocksDBBasedMap.valueStream().collect(Collectors.toList());
cntSize = 0;
for (HoodieRecord value : values) {
assert recordKeys.contains(value.getRecordKey());
cntSize++;
}
assertEquals(recordKeys.size(), cntSize);
}
@Test
public void testSimpleInsertRandomAccess() throws IOException, URISyntaxException {
RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath);
List<String> recordKeys = setupMapWithRecords(rocksDBBasedMap, 100);
Random random = new Random();
for (int i = 0; i < recordKeys.size(); i++) {
String key = recordKeys.get(random.nextInt(recordKeys.size()));
assert rocksDBBasedMap.get(key) != null;
}
}
@Test
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath);
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
Set<String> recordKeys = new HashSet<>();
// insert generated records into the map
hoodieRecords.forEach(r -> {
rocksDBBasedMap.put(r.getRecordKey(), r);
recordKeys.add(r.getRecordKey());
});
// make sure records have spilled to disk
assertTrue(rocksDBBasedMap.sizeOfFileOnDiskInBytes() > 0);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = rocksDBBasedMap.iterator();
int cntSize = 0;
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
cntSize++;
assert recordKeys.contains(rec.getRecordKey());
}
assertEquals(recordKeys.size(), cntSize);
}
@Test
public void testSimpleUpsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath);
List<IndexedRecord> insertedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(insertedRecords, rocksDBBasedMap);
String oldCommitTime =
((GenericRecord) insertedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
// generate updates from inserts for first 50 keys / subset of keys
List<IndexedRecord> updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys.subList(0, 50),
SchemaTestUtil.generateHoodieTestRecords(0, 50), HoodieActiveTimeline.createNewInstantTime());
String newCommitTime =
((GenericRecord) updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
// perform upserts
List<String> updatedRecordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, rocksDBBasedMap);
// Upserted records (on disk) should have the latest commit time
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = rocksDBBasedMap.iterator();
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
try {
IndexedRecord indexedRecord = (IndexedRecord) rec.getData().getInsertValue(schema).get();
String latestCommitTime =
((GenericRecord) indexedRecord).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
assert recordKeys.contains(rec.getRecordKey()) || updatedRecordKeys.contains(rec.getRecordKey());
assertEquals(latestCommitTime, updatedRecordKeys.contains(rec.getRecordKey()) ? newCommitTime : oldCommitTime);
} catch (IOException io) {
throw new UncheckedIOException(io);
}
}
}
@Test
public void testPutAll() throws IOException, URISyntaxException {
RocksDbDiskMap<String, HoodieRecord> rocksDBBasedMap = new RocksDbDiskMap<>(basePath);
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<String, HoodieRecord> recordMap = new HashMap<>();
iRecords.forEach(r -> {
String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord value = new HoodieRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r)));
recordMap.put(key, value);
});
rocksDBBasedMap.putAll(recordMap);
// make sure records have spilled to disk
assertTrue(rocksDBBasedMap.sizeOfFileOnDiskInBytes() > 0);
// make sure all added records are present
for (Map.Entry<String, HoodieRecord> entry : rocksDBBasedMap.entrySet()) {
assertTrue(recordMap.containsKey(entry.getKey()));
}
}
@Test
public void testSimpleRemove() throws IOException, URISyntaxException {
RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath);
List<String> recordKeys = setupMapWithRecords(rocksDBBasedMap, 100);
List<String> deleteKeys = recordKeys.subList(0, 10);
for (String deleteKey : deleteKeys) {
assert rocksDBBasedMap.remove(deleteKey) != null;
assert rocksDBBasedMap.get(deleteKey) == null;
}
}
private List<String> setupMapWithRecords(RocksDbDiskMap rocksDBBasedMap, int numRecords) throws IOException, URISyntaxException {
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, numRecords);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, rocksDBBasedMap);
// Ensure the number of records is correct
assertEquals(rocksDBBasedMap.size(), recordKeys.size());
// make sure records have spilled to disk
assertTrue(rocksDBBasedMap.sizeOfFileOnDiskInBytes() > 0);
return recordKeys;
}
}