1
0

Fixing behavior of Merge/CreateHandle for invalid/wrong schema records

This commit is contained in:
Nishith Agarwal
2019-01-22 13:04:15 -08:00
committed by vinoth chandar
parent 994d42d307
commit 7985eb72b5
8 changed files with 86 additions and 73 deletions

View File

@@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult;
import java.util.List;
import java.util.Optional;
import org.apache.avro.generic.IndexedRecord;
@@ -55,13 +56,13 @@ public class TestBoundedInMemoryExecutor {
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> consumer =
new BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer>() {
BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
new BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
private int count = 0;
@Override
protected void consumeOneRecord(Tuple2<HoodieRecord, Optional<IndexedRecord>> record) {
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> record) {
count++;
}

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer;
import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer;
import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@@ -78,9 +79,8 @@ public class TestBoundedInMemoryQueue {
public void testRecordReading() throws Exception {
final int numRecords = 128;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
final BoundedInMemoryQueue<HoodieRecord,
Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue = new BoundedInMemoryQueue(FileUtils.ONE_KB,
getTransformFunction(HoodieTestDataGenerator.avroSchema));
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> resFuture =
executorService.submit(() -> {
@@ -94,12 +94,12 @@ public class TestBoundedInMemoryQueue {
final HoodieRecord originalRecord = originalRecordIterator.next();
final Optional<IndexedRecord> originalInsertValue = originalRecord.getData()
.getInsertValue(HoodieTestDataGenerator.avroSchema);
final Tuple2<HoodieRecord, Optional<IndexedRecord>> payload = queue.iterator().next();
final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
// Ensure that record ordering is guaranteed.
Assert.assertEquals(originalRecord, payload._1());
Assert.assertEquals(originalRecord, payload.record);
// cached insert value matches the expected insert value.
Assert.assertEquals(originalInsertValue,
payload._1().getData().getInsertValue(HoodieTestDataGenerator.avroSchema));
payload.record.getData().getInsertValue(HoodieTestDataGenerator.avroSchema));
recordsRead++;
}
Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext());
@@ -119,7 +119,7 @@ public class TestBoundedInMemoryQueue {
final int numProducers = 40;
final List<List<HoodieRecord>> recs = new ArrayList<>();
final BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue =
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Record Key to <Producer Index, Rec Index within a producer>
@@ -143,7 +143,7 @@ public class TestBoundedInMemoryQueue {
if (i % 2 == 0) {
producers.add(new IteratorBasedQueueProducer<>(r.iterator()));
} else {
producers.add(new FunctionBasedQueueProducer<HoodieRecord>((buf) -> {
producers.add(new FunctionBasedQueueProducer<>((buf) -> {
Iterator<HoodieRecord> itr = r.iterator();
while (itr.hasNext()) {
try {
@@ -185,8 +185,8 @@ public class TestBoundedInMemoryQueue {
// Read recs and ensure we have covered all producer recs.
while (queue.iterator().hasNext()) {
final Tuple2<HoodieRecord, Optional<IndexedRecord>> payload = queue.iterator().next();
final HoodieRecord rec = payload._1();
final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
final HoodieRecord rec = payload.record;
Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
Integer lastSeenPos = lastSeenMap.get(producerPos._1());
countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1);
@@ -212,12 +212,13 @@ public class TestBoundedInMemoryQueue {
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
// maximum number of records to keep in memory.
final int recordLimit = 5;
final SizeEstimator<Tuple2<HoodieRecord, Optional<IndexedRecord>>> sizeEstimator =
final SizeEstimator<HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator =
new DefaultSizeEstimator<>();
final long objSize = sizeEstimator.sizeEstimate(
getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0)));
HoodieInsertValueGenResult<HoodieRecord> payload = getTransformFunction(HoodieTestDataGenerator.avroSchema)
.apply(hoodieRecords.get(0));
final long objSize = sizeEstimator.sizeEstimate(payload);
final long memoryLimitInBytes = recordLimit * objSize;
final BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue =
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
new BoundedInMemoryQueue(memoryLimitInBytes,
getTransformFunction(HoodieTestDataGenerator.avroSchema));
@@ -236,8 +237,8 @@ public class TestBoundedInMemoryQueue {
Assert.assertEquals(recordLimit - 1, queue.samplingRecordCounter.get());
// try to read 2 records.
Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next()._1());
Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next()._1());
Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next().record);
Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next().record);
// waiting for permits to expire.
while (!isQueueFull(queue.rateLimiter)) {
@@ -263,8 +264,9 @@ public class TestBoundedInMemoryQueue {
final SizeEstimator<Tuple2<HoodieRecord, Optional<IndexedRecord>>> sizeEstimator =
new DefaultSizeEstimator<>();
// queue memory limit
final long objSize = sizeEstimator.sizeEstimate(
getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0)));
HoodieInsertValueGenResult<HoodieRecord> payload = getTransformFunction(HoodieTestDataGenerator.avroSchema)
.apply(hoodieRecords.get(0));
final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue));
final long memoryLimitInBytes = 4 * objSize;
// first let us throw exception from queueIterator reader and test that queueing thread