1
0

Added support for Disk Spillable Compaction to prevent OOM issues

This commit is contained in:
Nishith Agarwal
2017-12-06 13:11:27 -08:00
committed by vinoth chandar
parent d495484399
commit 6fec9655a8
18 changed files with 1487 additions and 86 deletions

View File

@@ -16,12 +16,6 @@
package com.uber.hoodie.common.table.log;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
@@ -41,15 +35,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import com.uber.hoodie.common.util.collection.DiskBasedMap;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -64,6 +50,23 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@SuppressWarnings("Duplicates")
public class HoodieLogFormatTest {
@@ -490,7 +493,7 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
schema, "100", 10240L);
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -554,9 +557,8 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "102");
assertEquals("We read 200 records from 2 write batches", 200,
scanner.getTotalLogRecords());
schema, "102", 10240L);
assertEquals("We only read 200 records, but only 200 of them are valid", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
@@ -635,9 +637,8 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "103");
assertEquals("We would read 200 records", 200,
scanner.getTotalLogRecords());
schema, "103", 10240L);
assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
@@ -700,9 +701,8 @@ public class HoodieLogFormatTest {
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "102");
assertEquals("We still would read 200 records", 200,
scanner.getTotalLogRecords());
schema, "102", 10240L);
assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 150, readKeys.size());
@@ -720,7 +720,7 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(commandBlock);
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101");
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200,
readKeys.size());
@@ -783,12 +783,10 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
// all data must be rolled back before merge
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "100", 10240L);
assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>();
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -841,7 +839,7 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "100");
allLogFiles, schema, "100", 10240L);
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@@ -875,7 +873,7 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "100");
allLogFiles, schema, "100", 10240L);
assertEquals("We still would read 100 records", 100,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
@@ -931,7 +929,7 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "101");
allLogFiles, schema, "101", 10240L);
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@@ -1014,8 +1012,9 @@ public class HoodieLogFormatTest {
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "101");
allLogFiles, schema, "101", 10240L);
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
}

View File

@@ -18,6 +18,7 @@ package com.uber.hoodie.common.util;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
@@ -35,6 +36,7 @@ import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -53,7 +55,7 @@ public class SchemaTestUtil {
}
private static List<IndexedRecord> toRecords(Schema writerSchema, Schema readerSchema, int from,
int limit) throws IOException, URISyntaxException {
int limit) throws IOException, URISyntaxException {
GenericDatumReader<IndexedRecord> reader =
new GenericDatumReader<>(writerSchema, readerSchema);
// Required to register the necessary JAR:// file system
@@ -93,12 +95,29 @@ public class SchemaTestUtil {
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
throws IOException, URISyntaxException {
List<IndexedRecord> records = generateTestRecords(from, limit);
String commitTime = HoodieActiveTimeline.createNewCommitTime();
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
return records.stream()
.map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema))
.map(p -> {
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
return p;
}).collect(
Collectors.toList());
}
public static List<IndexedRecord> updateHoodieTestRecords(List<String> oldRecordKeys, List<IndexedRecord> newRecords,
String commitTime)
throws IOException, URISyntaxException {
return newRecords.stream()
.map(p -> {
((GenericRecord)p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0));
((GenericRecord)p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
((GenericRecord)p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
return p;
}).collect(
Collectors.toList());
@@ -121,7 +140,7 @@ public class SchemaTestUtil {
}
public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber,
String commitTime, String fileId) throws IOException {
String commitTime, String fileId) throws IOException {
TestRecord record = new TestRecord(commitTime, recordNumber, fileId);
MercifulJsonConverter converter = new MercifulJsonConverter(schema);
return converter.convert(record.toJsonString());

View File

@@ -0,0 +1,47 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
public class SpillableMapTestUtils {
public static List<String> upsertRecords(List<IndexedRecord> iRecords,
Map<String, HoodieRecord<? extends HoodieRecordPayload>> records) {
List<String> recordKeys = new ArrayList<>();
iRecords
.stream()
.forEach(r -> {
String key = ((GenericRecord)r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath = ((GenericRecord)r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
recordKeys.add(key);
records.put(key, new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) r))));
});
return recordKeys;
}
}

View File

@@ -0,0 +1,107 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.util.collection;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.junit.Test;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
public class TestDiskBasedMap {
@Test
public void testSimpleInsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty());
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
// 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());
}
}
@Test
public void testSimpleUpsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty());
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
// perform some inserts
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
long fileSize = records.sizeOfFileOnDiskInBytes();
// make sure records have spilled to disk
assertTrue(fileSize > 0);
// generate updates from inserts
List<IndexedRecord> updatedRecords =
SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100),
HoodieActiveTimeline.createNewCommitTime());
String newCommitTime = ((GenericRecord)updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
// new commit time should be different
assertEquals(commitTime, newCommitTime);
// perform upserts
recordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, records);
// upserts should be appended to the existing file, hence increasing the sizeOfFile on disk
assertTrue(records.sizeOfFileOnDiskInBytes() > fileSize);
// Upserted records (on disk) should have the latest commit time
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
while(itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
assert recordKeys.contains(rec.getRecordKey());
try {
IndexedRecord indexedRecord = (IndexedRecord)rec.getData().getInsertValue(schema).get();
String latestCommitTime = ((GenericRecord)indexedRecord).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
assertEquals(latestCommitTime, newCommitTime);
} catch(IOException io) {
throw new UncheckedIOException(io);
}
}
}
}

View File

@@ -0,0 +1,245 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.util.collection;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class TestExternalSpillableMap {
private static final String FAILURE_OUTPUT_PATH = "/tmp/test_fail";
@Test
public void simpleInsertTest() throws IOException, URISyntaxException {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
HoodieAvroPayload.class.getName(), Optional.empty()); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
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());
}
}
@Test
public void testSimpleUpsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, schema,
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
while(itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
assert recordKeys.contains(rec.getRecordKey());
}
List<IndexedRecord> updatedRecords =
SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100),
HoodieActiveTimeline.createNewCommitTime());
// update records already inserted
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
// make sure we have records spilled to disk
assertTrue(records.getDiskBasedMapNumEntries() > 0);
// iterate over the updated records and compare the value from Map
updatedRecords.stream().forEach(record -> {
HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
try {
assertEquals(rec.getData().getInsertValue(schema).get(),record);
} catch(IOException io) {
throw new UncheckedIOException(io);
}
});
}
@Test
public void testAllMapOperations() throws IOException, URISyntaxException {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
HoodieAvroPayload.class.getName(), Optional.empty()); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
// insert a bunch of records so that values spill to disk too
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
IndexedRecord inMemoryRecord = iRecords.get(0);
String ikey = ((GenericRecord)inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String iPartitionPath = ((GenericRecord)inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord)inMemoryRecord)));
IndexedRecord onDiskRecord = iRecords.get(99);
String dkey = ((GenericRecord)onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String dPartitionPath = ((GenericRecord)onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord)onDiskRecord)));
// assert size
assert records.size() == 100;
// get should return the same HoodieKey and same value
assert inMemoryHoodieRecord.getKey().equals(records.get(ikey).getKey());
assert onDiskHoodieRecord.getKey().equals(records.get(dkey).getKey());
//assert inMemoryHoodieRecord.equals(records.get(ikey));
//assert onDiskHoodieRecord.equals(records.get(dkey));
// test contains
assertTrue(records.containsKey(ikey));
assertTrue(records.containsKey(dkey));
// test isEmpty
assertFalse(records.isEmpty());
// test containsAll
assertTrue(records.keySet().containsAll(recordKeys));
// remove (from inMemory and onDisk)
HoodieRecord removedRecord = records.remove(ikey);
assertTrue(removedRecord != null);
assertFalse(records.containsKey(ikey));
removedRecord = records.remove(dkey);
assertTrue(removedRecord != null);
assertFalse(records.containsKey(dkey));
// test clear
records.clear();
assertTrue(records.size() == 0);
}
@Test(expected = IOException.class)
public void simpleTestWithException() throws IOException, URISyntaxException {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
while(itr.hasNext()) {
throw new IOException("Testing failures...");
}
}
@Test
public void simpleTestWithExceptionValidateFileIsRemoved() throws Exception {
File file = new File(FAILURE_OUTPUT_PATH);
assertFalse(file.exists());
}
@Test
public void testDataCorrectnessInMapAndDisk() throws IOException, URISyntaxException {
Schema schema = SchemaTestUtil.getSimpleSchema();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(schema),
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk
while(records.getDiskBasedMapNumEntries() < 1) {
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
recordKeys.addAll(SpillableMapTestUtils.upsertRecords(iRecords, records));
}
// Get a record from the in-Memory map
String key = recordKeys.get(0);
HoodieRecord record = records.get(key);
List<IndexedRecord> recordsToUpdate = new ArrayList<>();
schema = HoodieAvroUtils.addMetadataFields(schema);
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
String newCommitTime = HoodieActiveTimeline.createNewCommitTime();
List<String> keysToBeUpdated = new ArrayList<>();
keysToBeUpdated.add(key);
// Update the commitTime for this record
List<IndexedRecord> updatedRecords =
SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime);
// Upsert this updated record
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
// The record returned for this key should have the updated commitTime
assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
// Get a record from the disk based map
key = recordKeys.get(recordKeys.size() - 1);
record = records.get(key);
recordsToUpdate = new ArrayList<>();
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
newCommitTime = HoodieActiveTimeline.createNewCommitTime();
keysToBeUpdated = new ArrayList<>();
keysToBeUpdated.add(key);
// Update the commitTime for this record
updatedRecords =
SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime);
// Upsert this updated record
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
// The record returned for this key should have the updated commitTime
assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
}
// TODO : come up with a performance eval test for spillableMap
@Test
public void testLargeInsertUpsert() {
}
}