Making ExternalSpillableMap generic for any datatype
- Introduced concept of converters to be able to serde generic datatype for SpillableMap - Fixed/Added configs to Hoodie Configs - Changed HoodieMergeHandle to start using SpillableMap
This commit is contained in:
committed by
vinoth chandar
parent
fa787ab5ab
commit
987f5d6b96
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* Copyright (c) 2018 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.model;
|
||||
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
public class AvroBinaryTestPayload implements HoodieRecordPayload {
|
||||
|
||||
private final byte[] recordBytes;
|
||||
|
||||
public AvroBinaryTestPayload(Optional<GenericRecord> record) {
|
||||
|
||||
try {
|
||||
if (record.isPresent()) {
|
||||
recordBytes = HoodieAvroUtils.avroToBytes(record.get());
|
||||
} else {
|
||||
recordBytes = new byte[0];
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to convert payload to bytes");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecordPayload preCombine(HoodieRecordPayload another) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
|
||||
return getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
|
||||
}
|
||||
}
|
||||
@@ -17,9 +17,13 @@
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.avro.MercifulJsonConverter;
|
||||
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.exception.HoodieIOException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
@@ -32,9 +36,11 @@ import java.nio.file.Paths;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
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;
|
||||
@@ -109,21 +115,63 @@ public class SchemaTestUtil {
|
||||
|
||||
}
|
||||
|
||||
public static List<HoodieRecord> generateHoodieTestRecords(int from, int limit, Schema schema)
|
||||
throws IOException, URISyntaxException {
|
||||
List<IndexedRecord> records = generateTestRecords(from, limit);
|
||||
return records.stream()
|
||||
.map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, schema))
|
||||
.map(p -> convertToHoodieRecords(p,
|
||||
UUID.randomUUID().toString(), "000/00/00")).collect(
|
||||
Collectors.toList());
|
||||
}
|
||||
|
||||
private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) {
|
||||
return new HoodieRecord<>(new HoodieKey(key, partitionPath),
|
||||
new HoodieAvroPayload(Optional.of((GenericRecord) iRecord)));
|
||||
}
|
||||
|
||||
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);
|
||||
((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());
|
||||
|
||||
}
|
||||
|
||||
public static List<HoodieRecord> generateHoodieTestRecordsWithoutHoodieMetadata(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
|
||||
List<IndexedRecord> iRecords = generateTestRecords(from, limit);
|
||||
return iRecords
|
||||
.stream()
|
||||
.map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static List<HoodieRecord> updateHoodieTestRecordsWithoutHoodieMetadata(List<HoodieRecord> oldRecords, Schema schema,
|
||||
String fieldNameToUpdate, String newValue)
|
||||
throws IOException, URISyntaxException {
|
||||
return oldRecords
|
||||
.stream()
|
||||
.map(r -> {
|
||||
try {
|
||||
GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get();
|
||||
rec.put(fieldNameToUpdate, newValue);
|
||||
return new HoodieRecord<>(r.getKey(),
|
||||
new HoodieAvroPayload(Optional.of(rec)));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to get data from hoodie record", io);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static Schema getEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser()
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
|
||||
|
||||
@@ -36,8 +36,8 @@ public class SpillableMapTestUtils {
|
||||
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();
|
||||
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))));
|
||||
|
||||
@@ -16,45 +16,83 @@
|
||||
|
||||
package com.uber.hoodie.common.util.collection;
|
||||
|
||||
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
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.model.AvroBinaryTestPayload;
|
||||
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 com.uber.hoodie.common.util.SpillableMapUtils;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
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());
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(Optional.empty(),
|
||||
new StringConverter(), new HoodieRecordConverter(schema, payloadClazz));
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
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()) {
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
oRecords.add(rec);
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
|
||||
Schema schema = getSimpleSchema();
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(Optional.empty(),
|
||||
new StringConverter(), new HoodieRecordConverter(schema, payloadClazz));
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil
|
||||
.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
|
||||
Set<String> recordKeys = new HashSet<>();
|
||||
// insert generated records into the map
|
||||
hoodieRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
recordKeys.add(r.getRecordKey());
|
||||
});
|
||||
// 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());
|
||||
@@ -64,10 +102,14 @@ public class TestDiskBasedMap {
|
||||
@Test
|
||||
public void testSimpleUpsert() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty());
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(Optional.empty(),
|
||||
new StringConverter(), new HoodieRecordConverter(schema, payloadClazz));
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
String commitTime = ((GenericRecord) iRecords.get(0))
|
||||
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
// perform some inserts
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
|
||||
@@ -77,9 +119,11 @@ public class TestDiskBasedMap {
|
||||
|
||||
// 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();
|
||||
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);
|
||||
@@ -92,16 +136,75 @@ public class TestDiskBasedMap {
|
||||
|
||||
// Upserted records (on disk) should have the latest commit time
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
while(itr.hasNext()) {
|
||||
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();
|
||||
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) {
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSizeEstimator() throws IOException, URISyntaxException {
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
|
||||
// Test sizeEstimator without hoodie metadata fields
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
|
||||
long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Test sizeEstimator with hoodie metadata fields
|
||||
schema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Following tests payloads without an Avro Schema in the Record
|
||||
|
||||
// Test sizeEstimator without hoodie metadata fields and without schema object in the payload
|
||||
schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName()));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Test sizeEstimator with hoodie metadata fields and without schema object in the payload
|
||||
final Schema simpleSchemaWithMetadata = HoodieAvroUtils
|
||||
.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Optional
|
||||
.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
|
||||
.collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName()));
|
||||
assertTrue(payloadSize > 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSizeEstimatorPerformance() throws IOException, URISyntaxException {
|
||||
// Test sizeEstimatorPerformance with simpleSchema
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
HoodieRecordConverter converter =
|
||||
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName());
|
||||
HoodieRecord record = hoodieRecords.remove(0);
|
||||
long startTime = System.currentTimeMillis();
|
||||
SpillableMapUtils.computePayloadSize(record, converter);
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
assertTrue(timeTaken < 100);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,6 +16,10 @@
|
||||
|
||||
package com.uber.hoodie.common.util.collection;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
@@ -24,13 +28,8 @@ 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 com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
@@ -39,10 +38,12 @@ 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;
|
||||
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;
|
||||
|
||||
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
|
||||
public class TestExternalSpillableMap {
|
||||
@@ -51,17 +52,19 @@ public class TestExternalSpillableMap {
|
||||
|
||||
@Test
|
||||
public void simpleInsertTest() throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>
|
||||
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
|
||||
HoodieAvroPayload.class.getName(), Optional.empty()); //16B
|
||||
(16L, Optional.empty(), new StringConverter(),
|
||||
new HoodieRecordConverter(schema, payloadClazz)); //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()) {
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
oRecords.add(rec);
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
@@ -72,16 +75,18 @@ public class TestExternalSpillableMap {
|
||||
public void testSimpleUpsert() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>
|
||||
(16L, schema,
|
||||
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
|
||||
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
|
||||
new HoodieRecordConverter(schema, payloadClazz)); //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()) {
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
@@ -99,8 +104,8 @@ public class TestExternalSpillableMap {
|
||||
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) {
|
||||
assertEquals(rec.getData().getInsertValue(schema).get(), record);
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
});
|
||||
@@ -109,25 +114,28 @@ public class TestExternalSpillableMap {
|
||||
@Test
|
||||
public void testAllMapOperations() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>
|
||||
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
|
||||
HoodieAvroPayload.class.getName(), Optional.empty()); //16B
|
||||
(16L, Optional.empty(), new StringConverter(),
|
||||
new HoodieRecordConverter(schema, payloadClazz)); //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();
|
||||
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)));
|
||||
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();
|
||||
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)));
|
||||
new HoodieAvroPayload(Optional.of((GenericRecord) onDiskRecord)));
|
||||
// assert size
|
||||
assert records.size() == 100;
|
||||
// get should return the same HoodieKey and same value
|
||||
@@ -162,16 +170,19 @@ public class TestExternalSpillableMap {
|
||||
|
||||
@Test(expected = IOException.class)
|
||||
public void simpleTestWithException() throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>
|
||||
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
|
||||
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
|
||||
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
|
||||
new HoodieRecordConverter(schema, payloadClazz)); //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()) {
|
||||
while (itr.hasNext()) {
|
||||
throw new IOException("Testing failures...");
|
||||
}
|
||||
}
|
||||
@@ -183,17 +194,19 @@ public class TestExternalSpillableMap {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataCorrectnessInMapAndDisk() throws IOException, URISyntaxException {
|
||||
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
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
|
||||
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
|
||||
new HoodieRecordConverter(schema, payloadClazz)); //16B
|
||||
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
// Ensure we spill to disk
|
||||
while(records.getDiskBasedMapNumEntries() < 1) {
|
||||
while (records.getDiskBasedMapNumEntries() < 1) {
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
recordKeys.addAll(SpillableMapTestUtils.upsertRecords(iRecords, records));
|
||||
}
|
||||
@@ -202,7 +215,6 @@ public class TestExternalSpillableMap {
|
||||
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();
|
||||
@@ -235,7 +247,72 @@ public class TestExternalSpillableMap {
|
||||
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());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>
|
||||
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
|
||||
new HoodieRecordConverter(schema, payloadClazz)); //16B
|
||||
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
// Ensure we spill to disk
|
||||
while (records.getDiskBasedMapNumEntries() < 1) {
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 100);
|
||||
hoodieRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
recordKeys.add(r.getRecordKey());
|
||||
});
|
||||
}
|
||||
|
||||
// Get a record from the in-Memory map
|
||||
String key = recordKeys.get(0);
|
||||
HoodieRecord record = records.get(key);
|
||||
// Get the field we want to update
|
||||
String fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING)
|
||||
.findAny().get().name();
|
||||
// Use a new value to update this field
|
||||
String newValue = "update1";
|
||||
List<HoodieRecord> recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add(record);
|
||||
|
||||
List<HoodieRecord> updatedRecords =
|
||||
SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue);
|
||||
|
||||
// Upsert this updated record
|
||||
updatedRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
});
|
||||
GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
|
||||
// The record returned for this key should have the updated value for the field name
|
||||
assertEquals(gRecord.get(fieldName).toString(), newValue);
|
||||
|
||||
// Get a record from the disk based map
|
||||
key = recordKeys.get(recordKeys.size() - 1);
|
||||
record = records.get(key);
|
||||
// Get the field we want to update
|
||||
fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING)
|
||||
.findAny().get().name();
|
||||
// Use a new value to update this field
|
||||
newValue = "update2";
|
||||
recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add(record);
|
||||
|
||||
updatedRecords =
|
||||
SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue);
|
||||
|
||||
// Upsert this updated record
|
||||
updatedRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
});
|
||||
gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
|
||||
// The record returned for this key should have the updated value for the field name
|
||||
assertEquals(gRecord.get(fieldName).toString(), newValue);
|
||||
}
|
||||
|
||||
// TODO : come up with a performance eval test for spillableMap
|
||||
|
||||
Reference in New Issue
Block a user