1
0

Support union mode in HoodieRealtimeRecordReader for pure insert workloads

Also Replace BufferedIteratorPayload abstraction with function passing
This commit is contained in:
Balaji Varadarajan
2018-04-26 10:18:05 -07:00
committed by vinoth chandar
parent 93f345a032
commit dfc0c61eb7
44 changed files with 2545 additions and 1179 deletions

View File

@@ -1,209 +0,0 @@
/*
* 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.func;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.uber.hoodie.exception.HoodieException;
import java.util.Iterator;
import java.util.Optional;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.util.SizeEstimator;
/**
* Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It
* internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in
* buffer accordingly. This is done to ensure that we don't OOM.
*
* @param <I> input payload data type
* @param <O> output payload data type
*/
public class BufferedIterator<I, O> implements Iterator<O> {
// interval used for polling records in the queue.
public static final int RECORD_POLL_INTERVAL_SEC = 5;
// rate used for sampling records to determine avg record size in bytes.
public static final int RECORD_SAMPLING_RATE = 64;
// maximum records that will be cached
private static final int RECORD_CACHING_LIMIT = 128 * 1024;
private static Logger logger = LogManager.getLogger(BufferedIterator.class);
// It indicates number of records to cache. We will be using sampled record's average size to
// determine how many
// records we should cache and will change (increase/decrease) permits accordingly.
@VisibleForTesting
public final Semaphore rateLimiter = new Semaphore(1);
// used for sampling records with "RECORD_SAMPLING_RATE" frequency.
public final AtomicLong samplingRecordCounter = new AtomicLong(-1);
// internal buffer to cache buffered records.
private final LinkedBlockingQueue<Optional<O>> buffer = new
LinkedBlockingQueue<>();
// maximum amount of memory to be used for buffering records.
private final long bufferMemoryLimit;
// original iterator from where records are read for buffering.
private final Iterator<I> inputIterator;
// it holds the root cause of the exception in case either buffering records (reading from
// inputIterator) fails or
// thread reading records from buffer fails.
private final AtomicReference<Exception> hasFailed = new AtomicReference(null);
// used for indicating that all the records from buffer are read successfully.
private final AtomicBoolean isDone = new AtomicBoolean(false);
// indicates rate limit (number of records to cache). it is updated whenever there is a change
// in avg record size.
@VisibleForTesting
public int currentRateLimit = 1;
// indicates avg record size in bytes. It is updated whenever a new record is sampled.
@VisibleForTesting
public long avgRecordSizeInBytes = 0;
// indicates number of samples collected so far.
private long numSamples = 0;
// next record to be read from buffer.
private O nextRecord;
// Function to transform the input payload to the expected output payload
private Function<I, O> bufferedIteratorTransform;
public BufferedIterator(final Iterator<I> iterator, final long bufferMemoryLimit,
final Function<I, O> bufferedIteratorTransform) {
this.inputIterator = iterator;
this.bufferMemoryLimit = bufferMemoryLimit;
this.bufferedIteratorTransform = bufferedIteratorTransform;
}
@VisibleForTesting
public int size() {
return this.buffer.size();
}
// It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in
// bytes. It is used
// for determining how many maximum records to buffer. Based on change in avg size it may
// increase or decrease
// available permits.
private void adjustBufferSizeIfNeeded(final I record) throws InterruptedException {
if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) {
return;
}
final long recordSizeInBytes = SizeEstimator.estimate(record);
final long newAvgRecordSizeInBytes = Math
.max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1));
final int newRateLimit = (int) Math
.min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes));
// If there is any change in number of records to cache then we will either release (if it increased) or acquire
// (if it decreased) to adjust rate limiting to newly computed value.
if (newRateLimit > currentRateLimit) {
rateLimiter.release(newRateLimit - currentRateLimit);
} else if (newRateLimit < currentRateLimit) {
rateLimiter.acquire(currentRateLimit - newRateLimit);
}
currentRateLimit = newRateLimit;
avgRecordSizeInBytes = newAvgRecordSizeInBytes;
numSamples++;
}
// inserts record into internal buffer. It also fetches insert value from the record to offload
// computation work on to
// buffering thread.
private void insertRecord(I t) throws Exception {
rateLimiter.acquire();
adjustBufferSizeIfNeeded(t);
// We are retrieving insert value in the record buffering thread to offload computation
// around schema validation
// and record creation to it.
final O payload = bufferedIteratorTransform.apply(t);
buffer.put(Optional.of(payload));
}
private void readNextRecord() {
rateLimiter.release();
Optional<O> newRecord;
while (true) {
try {
throwExceptionIfFailed();
newRecord = buffer.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS);
if (newRecord != null) {
break;
}
} catch (InterruptedException e) {
logger.error("error reading records from BufferedIterator", e);
throw new HoodieException(e);
}
}
if (newRecord.isPresent()) {
this.nextRecord = newRecord.get();
} else {
// We are done reading all the records from internal iterator.
this.isDone.set(true);
this.nextRecord = null;
}
}
public void startBuffering() throws Exception {
logger.info("starting to buffer records");
try {
while (inputIterator.hasNext()) {
// We need to stop buffering if buffer-reader has failed and exited.
throwExceptionIfFailed();
insertRecord(inputIterator.next());
}
// done buffering records notifying buffer-reader.
buffer.put(Optional.empty());
} catch (Exception e) {
logger.error("error buffering records", e);
// Used for notifying buffer-reader thread of the failed operation.
markAsFailed(e);
throw e;
}
logger.info("finished buffering records");
}
@Override
public boolean hasNext() {
if (this.nextRecord == null && !this.isDone.get()) {
readNextRecord();
}
return !this.isDone.get();
}
@Override
public O next() {
Preconditions.checkState(hasNext() && this.nextRecord != null);
final O ret = this.nextRecord;
this.nextRecord = null;
return ret;
}
private void throwExceptionIfFailed() {
if (this.hasFailed.get() != null) {
throw new HoodieException("operation has failed", this.hasFailed.get());
}
}
public void markAsFailed(Exception e) {
this.hasFailed.set(e);
// release the permits so that if the buffering thread is waiting for permits then it will
// get it.
this.rateLimiter.release(RECORD_CACHING_LIMIT + 1);
}
}

View File

@@ -1,89 +0,0 @@
/*
* 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.func;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import java.util.Iterator;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.function.Function;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$;
/**
* Executor for a BufferedIterator operation. This class takes as input the input iterator which
* needs to be buffered, the runnable function that needs to be executed in the reader thread and
* return the transformed output based on the writer function
*/
public class BufferedIteratorExecutor<I, O, E> {
private static Logger logger = LogManager.getLogger(BufferedIteratorExecutor.class);
// Executor service used for launching writer thread.
final ExecutorService writerService;
// Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
final BufferedIterator<I, O> bufferedIterator;
// Need to set current spark thread's TaskContext into newly launched thread so that new
// thread can access
// TaskContext properties.
final TaskContext sparkThreadTaskContext;
public BufferedIteratorExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
final Function<I, O> bufferedIteratorTransform,
final ExecutorService writerService) {
this.sparkThreadTaskContext = TaskContext.get();
this.writerService = writerService;
this.bufferedIterator = new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(),
bufferedIteratorTransform);
}
/**
* Starts buffering and executing the writer function
*/
public Future<E> start(Function<BufferedIterator, E> writerFunction) {
try {
Future<E> future = writerService.submit(
() -> {
logger.info("starting hoodie writer thread");
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext
// properties.
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
try {
E result = writerFunction.apply(bufferedIterator);
logger.info("hoodie write is done; notifying reader thread");
return result;
} catch (Exception e) {
logger.error("error writing hoodie records", e);
bufferedIterator.markAsFailed(e);
throw e;
}
});
bufferedIterator.startBuffering();
return future;
} catch (Exception e) {
throw new HoodieException(e);
}
}
public boolean isRemaining() {
return bufferedIterator.hasNext();
}
}

View File

@@ -19,27 +19,25 @@ package com.uber.hoodie.func;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload;
import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieIOHandle;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.function.Function;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.spark.TaskContext;
import scala.Tuple2;
/**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
@@ -52,7 +50,6 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
private final String commitTime;
private final HoodieTable<T> hoodieTable;
private Set<String> partitionsCleaned;
private HoodieCreateHandle handle;
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) {
@@ -63,57 +60,68 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
this.hoodieTable = hoodieTable;
}
@Override
protected void start() {
}
/**
* Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
* expensive operations of transformation to the reader thread.
* @param schema
* @param <T>
* @return
*/
public static <T extends HoodieRecordPayload> Function<HoodieRecord<T>, AbstractBufferedIteratorPayload>
bufferedItrPayloadTransform(Schema schema) {
return (hoodieRecord) -> new HoodieRecordBufferedIteratorPayload(hoodieRecord, schema);
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>,
Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>> getTransformFunction(Schema schema) {
return hoodieRecord -> {
try {
return new Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>(hoodieRecord,
hoodieRecord.getData().getInsertValue(schema));
} catch (IOException e) {
throw new HoodieException(e);
}
};
}
@Override
protected void start() {
}
@Override
protected List<WriteStatus> computeNext() {
// Executor service used for launching writer thread.
final ExecutorService writerService = Executors.newFixedThreadPool(1);
BoundedInMemoryExecutor<HoodieRecord<T>,
Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> bufferedIteratorExecutor = null;
try {
Function<BufferedIterator, List<WriteStatus>> function = (bufferedIterator) -> {
List<WriteStatus> statuses = new LinkedList<>();
statuses.addAll(handleWrite(bufferedIterator));
return statuses;
};
BufferedIteratorExecutor<HoodieRecord<T>, AbstractBufferedIteratorPayload, List<WriteStatus>>
bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieConfig, inputItr,
bufferedItrPayloadTransform(HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)),
writerService);
Future<List<WriteStatus>> writerResult = bufferedIteratorExecutor.start(function);
final List<WriteStatus> result = writerResult.get();
final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig);
bufferedIteratorExecutor =
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr,
new InsertHandler(), getTransformFunction(schema));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result;
} catch (Exception e) {
throw new HoodieException(e);
} finally {
writerService.shutdownNow();
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
}
}
}
private List<WriteStatus> handleWrite(
final BufferedIterator<HoodieRecord<T>, AbstractBufferedIteratorPayload> bufferedIterator) {
List<WriteStatus> statuses = new ArrayList<>();
while (bufferedIterator.hasNext()) {
final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator
.next();
final HoodieRecord insertPayload = (HoodieRecord) payload.getInputPayload();
@Override
protected void end() {
}
/**
* Consumes stream of hoodie records from in-memory queue and
* writes to one or more create-handles
*/
private class InsertHandler extends
BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> {
private final List<WriteStatus> statuses = new ArrayList<>();
private HoodieCreateHandle handle;
@Override
protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) {
final HoodieRecord insertPayload = payload._1();
// clean up any partial failures
if (!partitionsCleaned
.contains(insertPayload.getPartitionPath())) {
if (!partitionsCleaned.contains(insertPayload.getPartitionPath())) {
// This insert task could fail multiple times, but Spark will faithfully retry with
// the same data again. Thus, before we open any files under a given partition, we
// first delete any files in the same partitionPath written by same Spark partition
@@ -127,33 +135,30 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath());
}
if (handle.canWrite(((HoodieRecord) payload.getInputPayload()))) {
if (handle.canWrite(payload._1())) {
// write the payload, if the handle has capacity
handle.write(insertPayload, (Optional<IndexedRecord>) payload.getOutputPayload(), payload.exception);
handle.write(insertPayload, payload._2());
} else {
// handle is full.
statuses.add(handle.close());
// Need to handle the rejected payload & open new handle
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath());
handle.write(insertPayload,
(Optional<IndexedRecord>) payload.getOutputPayload(),
payload.exception); // we should be able to write 1 payload.
handle.write(insertPayload, payload._2()); // we should be able to write 1 payload.
}
}
// If we exited out, because we ran out of records, just close the pending handle.
if (!bufferedIterator.hasNext()) {
@Override
protected void finish() {
if (handle != null) {
statuses.add(handle.close());
}
handle = null;
assert statuses.size() > 0;
}
assert statuses.size() > 0 && !bufferedIterator.hasNext(); // should never return empty statuses
return statuses;
}
@Override
protected void end() {
@Override
protected List<WriteStatus> getResult() {
return statuses;
}
}
}

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.Iterator;
@@ -23,7 +24,7 @@ import org.apache.parquet.hadoop.ParquetReader;
/**
* This class wraps a parquet reader and provides an iterator based api to
* read from a parquet file. This is used in {@link BufferedIterator}
* read from a parquet file. This is used in {@link BoundedInMemoryQueue}
*/
public class ParquetReaderIterator<T> implements Iterator<T> {

View File

@@ -0,0 +1,57 @@
/*
* Copyright (c) 2017 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.func;
import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer;
import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.util.Iterator;
import java.util.Optional;
import java.util.function.Function;
import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$;
public class SparkBoundedInMemoryExecutor<I, O, E> extends BoundedInMemoryExecutor<I, O, E> {
// Need to set current spark thread's TaskContext into newly launched thread so that new thread can access
// TaskContext properties.
final TaskContext sparkThreadTaskContext;
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
BoundedInMemoryQueueConsumer<O, E> consumer,
Function<I, O> bufferedIteratorTransform) {
this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform);
}
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig,
BoundedInMemoryQueueProducer<I> producer,
BoundedInMemoryQueueConsumer<O, E> consumer,
Function<I, O> bufferedIteratorTransform) {
super(hoodieConfig.getWriteBufferLimitBytes(), producer,
Optional.of(consumer), bufferedIteratorTransform);
this.sparkThreadTaskContext = TaskContext.get();
}
public void preExecute() {
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties.
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
}
}

View File

@@ -1,42 +0,0 @@
/*
* 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.func.payload;
/**
* @param <I> Input data type for BufferedIterator
* @param <O> Output data type for BufferedIterator
*/
public abstract class AbstractBufferedIteratorPayload<I, O> {
// input payload for iterator
protected I inputPayload;
// output payload for iterator, this is used in cases where the output payload is computed
// from the input payload and most of this computation is off-loaded to the reader
protected O outputPayload;
public AbstractBufferedIteratorPayload(I record) {
this.inputPayload = record;
}
public I getInputPayload() {
return inputPayload;
}
public O getOutputPayload() {
return outputPayload;
}
}

View File

@@ -1,31 +0,0 @@
/*
* 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.func.payload;
import org.apache.avro.generic.GenericRecord;
/**
* BufferedIteratorPayload that takes GenericRecord as input and GenericRecord as output
*/
public class GenericRecordBufferedIteratorPayload
extends AbstractBufferedIteratorPayload<GenericRecord, GenericRecord> {
public GenericRecordBufferedIteratorPayload(GenericRecord record) {
super(record);
this.outputPayload = record;
}
}

View File

@@ -1,47 +0,0 @@
/*
* 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.func.payload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
/**
* BufferedIteratorPayload that takes HoodieRecord as input and transforms to output Optional<IndexedRecord>
* @param <T>
*/
public class HoodieRecordBufferedIteratorPayload<T extends HoodieRecordPayload>
extends AbstractBufferedIteratorPayload<HoodieRecord<T>, Optional<IndexedRecord>> {
// It caches the exception seen while fetching insert value.
public Optional<Exception> exception = Optional.empty();
public HoodieRecordBufferedIteratorPayload(HoodieRecord record, Schema schema) {
super(record);
try {
this.outputPayload = record.getData().getInsertValue(schema);
} catch (Exception e) {
this.exception = Optional.of(e);
}
}
public Optional<Exception> getException() {
return exception;
}
}

View File

@@ -90,15 +90,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
/**
* Perform the actual writing of the given record into the backing file.
*/
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue,
Optional<Exception> getInsertValueException) {
public void write(HoodieRecord record, Optional<IndexedRecord> avroRecord) {
Optional recordMetadata = record.getData().getMetadata();
try {
// throws exception if there was any exception while fetching insert value
if (getInsertValueException.isPresent()) {
throw getInsertValueException.get();
}
Optional<IndexedRecord> avroRecord = insertValue;
if (avroRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
// update the new location of record, so we know where to find it next

View File

@@ -24,7 +24,9 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
@@ -143,7 +145,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
config.getSpillableMapBasePath(), new StringConverter(),
new HoodieRecordConverter(schema, config.getPayloadClass()));
new HoodieRecordConverter(schema, config.getPayloadClass()),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema));
} catch (IOException io) {
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
}

View File

@@ -28,7 +28,7 @@ import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner;
import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -115,7 +115,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.filterCompletedInstants().lastInstant().get().getTimestamp();
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime,
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
@@ -131,7 +131,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
Iterable<List<WriteStatus>> resultIterable = () -> result;
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
.map(s -> {
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getTotalRecordsToUpdate());
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
s.getStat().setPartitionPath(operation.getPartitionPath());

View File

@@ -33,17 +33,16 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.BufferedIterator;
import com.uber.hoodie.func.BufferedIteratorExecutor;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.func.ParquetReaderIterator;
import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload;
import com.uber.hoodie.func.payload.GenericRecordBufferedIteratorPayload;
import com.uber.hoodie.func.SparkBoundedInMemoryExecutor;
import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieMergeHandle;
import java.io.IOException;
@@ -58,9 +57,6 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -182,16 +178,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
}
/**
* Transformer function to help transform a GenericRecord. This transformer is used by BufferedIterator to offload
* some expensive operations of transformation to the reader thread.
*
*/
public static java.util.function.Function<GenericRecord, AbstractBufferedIteratorPayload>
bufferedItrPayloadTransform() {
return (genericRecord) -> new GenericRecordBufferedIteratorPayload(genericRecord);
}
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
String commitTime, String fileLoc)
throws IOException {
@@ -202,23 +188,19 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
ParquetReader<IndexedRecord> reader = AvroParquetReader.builder(upsertHandle.getOldFilePath())
.withConf(getHadoopConf()).build();
final ExecutorService writerService = Executors.newFixedThreadPool(1);
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
try {
java.util.function.Function<BufferedIterator, Void> runnableFunction = (bufferedIterator) -> {
handleWrite(bufferedIterator, upsertHandle);
return null;
};
BufferedIteratorExecutor<GenericRecord, AbstractBufferedIteratorPayload, Void> wrapper =
new BufferedIteratorExecutor(config, new ParquetReaderIterator(reader), bufferedItrPayloadTransform(),
writerService);
Future writerResult = wrapper.start(runnableFunction);
writerResult.get();
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
new UpdateHandler(upsertHandle), x -> x);
wrapper.execute();
} catch (Exception e) {
throw new HoodieException(e);
} finally {
reader.close();
upsertHandle.close();
writerService.shutdownNow();
if (null != wrapper) {
wrapper.shutdownNow();
}
}
}
@@ -231,15 +213,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
.iterator();
}
private void handleWrite(final BufferedIterator<GenericRecord, GenericRecord> bufferedIterator,
final HoodieMergeHandle upsertHandle) {
while (bufferedIterator.hasNext()) {
final GenericRecordBufferedIteratorPayload payload = (GenericRecordBufferedIteratorPayload) bufferedIterator
.next();
upsertHandle.write(payload.getOutputPayload());
}
}
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
Iterator<HoodieRecord<T>> recordItr) {
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
@@ -493,6 +466,32 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
UPDATE, INSERT
}
/**
* Consumer that dequeues records from queue and sends to Merge Handle
*/
private static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
private final HoodieMergeHandle upsertHandle;
private UpdateHandler(HoodieMergeHandle upsertHandle) {
this.upsertHandle = upsertHandle;
}
@Override
protected void consumeOneRecord(GenericRecord record) {
upsertHandle.write(record);
}
@Override
protected void finish() {
}
@Override
protected Void getResult() {
return null;
}
}
private static class PartitionCleanStat implements Serializable {
private final String partitionPath;

View File

@@ -16,39 +16,35 @@
package com.uber.hoodie.func;
import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.common.HoodieTestDataGenerator;
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 java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.function.Function;
import java.util.Optional;
import org.apache.avro.generic.IndexedRecord;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import scala.Tuple2;
public class TestBufferedIteratorExecutor {
public class TestBoundedInMemoryExecutor {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private ExecutorService executorService = null;
@Before
public void beforeTest() {
this.executorService = Executors.newFixedThreadPool(1);
}
private SparkBoundedInMemoryExecutor<HoodieRecord,
Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> executor = null;
@After
public void afterTest() {
if (this.executorService != null) {
this.executorService.shutdownNow();
this.executorService = null;
if (this.executor != null) {
this.executor.shutdownNow();
this.executor = null;
}
}
@@ -59,21 +55,32 @@ public class TestBufferedIteratorExecutor {
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
BufferedIteratorExecutor bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieWriteConfig,
hoodieRecords.iterator(), LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema),
executorService);
Function<BufferedIterator, Integer> function = (bufferedIterator) -> {
Integer count = 0;
while (bufferedIterator.hasNext()) {
count++;
bufferedIterator.next();
}
return count;
};
Future<Integer> future = bufferedIteratorExecutor.start(function);
BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> consumer =
new BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer>() {
private int count = 0;
@Override
protected void consumeOneRecord(Tuple2<HoodieRecord, Optional<IndexedRecord>> record) {
count++;
}
@Override
protected void finish() {
}
@Override
protected Integer getResult() {
return count;
}
};
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig,
hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.avroSchema));
int result = executor.execute();
// It should buffer and write 100 records
Assert.assertEquals((int) future.get(), 100);
Assert.assertEquals(result, 100);
// There should be no remaining records in the buffer
Assert.assertFalse(bufferedIteratorExecutor.isRemaining());
Assert.assertFalse(executor.isRemaining());
}
}

View File

@@ -0,0 +1,336 @@
/*
* 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.func;
import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.SizeEstimator;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue;
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 java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.Semaphore;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.FileUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import scala.Tuple2;
public class TestBoundedInMemoryQueue {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private ExecutorService executorService = null;
@Before
public void beforeTest() {
this.executorService = Executors.newFixedThreadPool(2);
}
@After
public void afterTest() {
if (this.executorService != null) {
this.executorService.shutdownNow();
this.executorService = null;
}
}
// Test to ensure that we are reading all records from queue iterator in the same order
// without any exceptions.
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
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));
// Produce
Future<Boolean> resFuture =
executorService.submit(() -> {
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
queue.close();
return true;
});
final Iterator<HoodieRecord> originalRecordIterator = hoodieRecords.iterator();
int recordsRead = 0;
while (queue.iterator().hasNext()) {
final HoodieRecord originalRecord = originalRecordIterator.next();
final Optional<IndexedRecord> originalInsertValue = originalRecord.getData()
.getInsertValue(HoodieTestDataGenerator.avroSchema);
final Tuple2<HoodieRecord, Optional<IndexedRecord>> payload = queue.iterator().next();
// Ensure that record ordering is guaranteed.
Assert.assertEquals(originalRecord, payload._1());
// cached insert value matches the expected insert value.
Assert.assertEquals(originalInsertValue,
payload._1().getData().getInsertValue(HoodieTestDataGenerator.avroSchema));
recordsRead++;
}
Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext());
// all the records should be read successfully.
Assert.assertEquals(numRecords, recordsRead);
// should not throw any exceptions.
resFuture.get();
}
/**
* Test to ensure that we are reading all records from queue iterator when we have multiple producers
*/
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testCompositeProducerRecordReading() throws Exception {
final int numRecords = 1000;
final int numProducers = 40;
final List<List<HoodieRecord>> recs = new ArrayList<>();
final BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue =
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Record Key to <Producer Index, Rec Index within a producer>
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
for (int i = 0; i < numProducers; i++) {
List<HoodieRecord> pRecs = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
int j = 0;
for (HoodieRecord r : pRecs) {
Assert.assertTrue(!keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j));
j++;
}
recs.add(pRecs);
}
List<BoundedInMemoryQueueProducer<HoodieRecord>> producers = new ArrayList<>();
for (int i = 0; i < recs.size(); i++) {
final List<HoodieRecord> r = recs.get(i);
// Alternate between pull and push based iterators
if (i % 2 == 0) {
producers.add(new IteratorBasedQueueProducer<>(r.iterator()));
} else {
producers.add(new FunctionBasedQueueProducer<HoodieRecord>((buf) -> {
Iterator<HoodieRecord> itr = r.iterator();
while (itr.hasNext()) {
try {
buf.insertRecord(itr.next());
} catch (Exception e) {
throw new HoodieException(e);
}
}
return true;
}));
}
}
final List<Future<Boolean>> futureList = producers.stream().map(producer -> {
return executorService.submit(() -> {
producer.produce(queue);
return true;
});
}).collect(Collectors.toList());
// Close queue
Future<Boolean> closeFuture = executorService.submit(() -> {
try {
for (Future f : futureList) {
f.get();
}
queue.close();
} catch (Exception e) {
throw new RuntimeException(e);
}
return true;
});
// Used to ensure that consumer sees the records generated by a single producer in FIFO order
Map<Integer, Integer> lastSeenMap = IntStream.range(0, numProducers).boxed()
.collect(Collectors.toMap(Function.identity(), x -> -1));
Map<Integer, Integer> countMap = IntStream.range(0, numProducers).boxed()
.collect(Collectors.toMap(Function.identity(), x -> 0));
// 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();
Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
Integer lastSeenPos = lastSeenMap.get(producerPos._1());
countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1);
lastSeenMap.put(producerPos._1(), lastSeenPos + 1);
// Ensure we are seeing the next record generated
Assert.assertEquals(lastSeenPos + 1, producerPos._2().intValue());
}
for (int i = 0; i < numProducers; i++) {
// Ensure we have seen all the records for each producers
Assert.assertEquals(Integer.valueOf(numRecords), countMap.get(i));
}
//Ensure Close future is done
closeFuture.get();
}
// Test to ensure that record queueing is throttled when we hit memory limit.
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testMemoryLimitForBuffering() throws Exception {
final int numRecords = 128;
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 =
new DefaultSizeEstimator<>();
final long objSize = sizeEstimator.sizeEstimate(
getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0)));
final long memoryLimitInBytes = recordLimit * objSize;
final BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue =
new BoundedInMemoryQueue(memoryLimitInBytes,
getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> resFuture = executorService.submit(() -> {
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
return true;
});
// waiting for permits to expire.
while (!isQueueFull(queue.rateLimiter)) {
Thread.sleep(10);
}
Assert.assertEquals(0, queue.rateLimiter.availablePermits());
Assert.assertEquals(recordLimit, queue.currentRateLimit);
Assert.assertEquals(recordLimit, queue.size());
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());
// waiting for permits to expire.
while (!isQueueFull(queue.rateLimiter)) {
Thread.sleep(10);
}
// No change is expected in rate limit or number of queued records. We only expect
// queueing thread to read
// 2 more records into the queue.
Assert.assertEquals(0, queue.rateLimiter.availablePermits());
Assert.assertEquals(recordLimit, queue.currentRateLimit);
Assert.assertEquals(recordLimit, queue.size());
Assert.assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get());
}
// Test to ensure that exception in either queueing thread or BufferedIterator-reader thread
// is propagated to
// another thread.
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testException() throws Exception {
final int numRecords = 256;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
final SizeEstimator<Tuple2<HoodieRecord, Optional<IndexedRecord>>> sizeEstimator =
new DefaultSizeEstimator<>();
// queue memory limit
final long objSize = sizeEstimator.sizeEstimate(
getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0)));
final long memoryLimitInBytes = 4 * objSize;
// first let us throw exception from queueIterator reader and test that queueing thread
// stops and throws
// correct exception back.
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue1 =
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> resFuture = executorService.submit(() -> {
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue1);
return true;
});
// waiting for permits to expire.
while (!isQueueFull(queue1.rateLimiter)) {
Thread.sleep(10);
}
// notify queueing thread of an exception and ensure that it exits.
final Exception e = new Exception("Failing it :)");
queue1.markAsFailed(e);
try {
resFuture.get();
Assert.fail("exception is expected");
} catch (ExecutionException e1) {
Assert.assertEquals(HoodieException.class, e1.getCause().getClass());
Assert.assertEquals(e, e1.getCause().getCause());
}
// second let us raise an exception while doing record queueing. this exception should get
// propagated to
// queue iterator reader.
final RuntimeException expectedException = new RuntimeException("failing record reading");
final Iterator<HoodieRecord> mockHoodieRecordsIterator = mock(Iterator.class);
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue2 =
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> res = executorService.submit(() -> {
try {
new IteratorBasedQueueProducer<>(mockHoodieRecordsIterator).produce(queue2);
} catch (Exception ex) {
queue2.markAsFailed(ex);
throw ex;
}
return true;
});
try {
queue2.iterator().hasNext();
Assert.fail("exception is expected");
} catch (Exception e1) {
Assert.assertEquals(expectedException, e1.getCause());
}
// queueing thread should also have exited. make sure that it is not running.
try {
res.get();
Assert.fail("exception is expected");
} catch (ExecutionException e2) {
Assert.assertEquals(expectedException, e2.getCause());
}
}
private boolean isQueueFull(Semaphore rateLimiter) {
return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads());
}
}

View File

@@ -1,203 +0,0 @@
/*
* 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.func;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload;
import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.Semaphore;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.FileUtils;
import org.apache.spark.util.SizeEstimator;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TestBufferedIterator {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private ExecutorService recordReader = null;
@Before
public void beforeTest() {
this.recordReader = Executors.newFixedThreadPool(1);
}
@After
public void afterTest() {
if (this.recordReader != null) {
this.recordReader.shutdownNow();
this.recordReader = null;
}
}
// Test to ensure that we are reading all records from buffered iterator in the same order
// without any exceptions.
@Test(timeout = 60000)
public void testRecordReading() throws IOException, ExecutionException, InterruptedException {
final int numRecords = 128;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB,
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result = recordReader.submit(() -> {
bufferedIterator.startBuffering();
return true;
});
final Iterator<HoodieRecord> originalRecordIterator = hoodieRecords.iterator();
int recordsRead = 0;
while (bufferedIterator.hasNext()) {
final HoodieRecord originalRecord = originalRecordIterator.next();
final Optional<IndexedRecord> originalInsertValue = originalRecord.getData()
.getInsertValue(HoodieTestDataGenerator.avroSchema);
final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator.next();
// Ensure that record ordering is guaranteed.
Assert.assertEquals(originalRecord, payload.getInputPayload());
// cached insert value matches the expected insert value.
Assert.assertEquals(originalInsertValue,
((HoodieRecord) payload.getInputPayload()).getData().getInsertValue(HoodieTestDataGenerator.avroSchema));
recordsRead++;
}
Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext());
// all the records should be read successfully.
Assert.assertEquals(numRecords, recordsRead);
// should not throw any exceptions.
Assert.assertTrue(result.get());
}
// Test to ensure that record buffering is throttled when we hit memory limit.
@Test(timeout = 60000)
public void testMemoryLimitForBuffering() throws IOException, InterruptedException {
final int numRecords = 128;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
// maximum number of records to keep in memory.
final int recordLimit = 5;
final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0));
final BufferedIterator<HoodieRecord, AbstractBufferedIteratorPayload> bufferedIterator =
new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes,
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result = recordReader.submit(() -> {
bufferedIterator.startBuffering();
return true;
});
// waiting for permits to expire.
while (!isQueueFull(bufferedIterator.rateLimiter)) {
Thread.sleep(10);
}
Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits());
Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit);
Assert.assertEquals(recordLimit, bufferedIterator.size());
Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get());
// try to read 2 records.
Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().getInputPayload());
Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().getInputPayload());
// waiting for permits to expire.
while (!isQueueFull(bufferedIterator.rateLimiter)) {
Thread.sleep(10);
}
// No change is expected in rate limit or number of buffered records. We only expect
// buffering thread to read
// 2 more records into the buffer.
Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits());
Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit);
Assert.assertEquals(recordLimit, bufferedIterator.size());
Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get());
}
// Test to ensure that exception in either buffering thread or BufferedIterator-reader thread
// is propagated to
// another thread.
@Test(timeout = 60000)
public void testException() throws IOException, InterruptedException {
final int numRecords = 256;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
// buffer memory limit
final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0));
// first let us throw exception from bufferIterator reader and test that buffering thread
// stops and throws
// correct exception back.
BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes,
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result = recordReader.submit(() -> {
bufferedIterator1.startBuffering();
return true;
});
// waiting for permits to expire.
while (!isQueueFull(bufferedIterator1.rateLimiter)) {
Thread.sleep(10);
}
// notify buffering thread of an exception and ensure that it exits.
final Exception e = new Exception("Failing it :)");
bufferedIterator1.markAsFailed(e);
try {
result.get();
Assert.fail("exception is expected");
} catch (ExecutionException e1) {
Assert.assertEquals(HoodieException.class, e1.getCause().getClass());
Assert.assertEquals(e, e1.getCause().getCause());
}
// second let us raise an exception while doing record buffering. this exception should get
// propagated to
// buffered iterator reader.
final RuntimeException expectedException = new RuntimeException("failing record reading");
final Iterator<HoodieRecord> mockHoodieRecordsIterator = mock(Iterator.class);
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes,
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result2 = recordReader.submit(() -> {
bufferedIterator2.startBuffering();
return true;
});
try {
bufferedIterator2.hasNext();
Assert.fail("exception is expected");
} catch (Exception e1) {
Assert.assertEquals(expectedException, e1.getCause());
}
// buffering thread should also have exited. make sure that it is not running.
try {
result2.get();
Assert.fail("exception is expected");
} catch (ExecutionException e2) {
Assert.assertEquals(expectedException, e2.getCause());
}
}
private boolean isQueueFull(Semaphore rateLimiter) {
return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads());
}
}