/** * Construct BoundedInMemoryQueue with passed in size estimator * * @param memoryLimit MemoryLimit in bytes * @param transformFunction Transformer Function to convert input payload type to stored payload type * @param payloadSizeEstimator Payload Size Estimator */ public BoundedInMemoryQueue( final long memoryLimit, final Function<I, O> transformFunction, final SizeEstimator<O> payloadSizeEstimator) { this.memoryLimit = memoryLimit; this.transformFunction = transformFunction; this.payloadSizeEstimator = payloadSizeEstimator; this.iterator = new QueueIterator(); }
/** * API to de-queue entries to memory bounded queue * * @param queue In Memory bounded queue */ public O consume(BoundedInMemoryQueue<?, I> queue) throws Exception { Iterator<I> iterator = queue.iterator(); while (iterator.hasNext()) { consumeOneRecord(iterator.next()); } // Notifies done finish(); return getResult(); }
public boolean isRemaining() { return queue.iterator().hasNext(); }
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema)); producers.add(new IteratorBasedQueueProducer<>(r.iterator())); } else { producers.add(new FunctionBasedQueueProducer<>((buf) -> { Iterator<HoodieRecord> itr = r.iterator(); while (itr.hasNext()) { try { buf.insertRecord(itr.next()); } catch (Exception e) { throw new HoodieException(e); producer.produce(queue); return true; }); f.get(); queue.close(); } catch (Exception e) { throw new RuntimeException(e); while (queue.iterator().hasNext()) { final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next(); final HoodieRecord rec = payload.record; Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue = new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema)); 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 HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next(); recordsRead++; Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext());
this.executor = new BoundedInMemoryExecutor<>(getMaxCompactionMemoryInBytes(), getParallelProducers(), Optional.empty(), x -> x, new DefaultSizeEstimator<>()); this.iterator = this.executor.getQueue().iterator(); this.logRecordScanner = new HoodieUnMergedLogRecordScanner( FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), this.executor.getQueue().insertRecord(aWritable); }); this.executor.startProducers();
final long memoryLimitInBytes = recordLimit * objSize; final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); return true; }); Assert.assertEquals(recordLimit, queue.size()); Assert.assertEquals(recordLimit - 1, queue.samplingRecordCounter.get()); Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next().record); Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next().record); Assert.assertEquals(recordLimit, queue.size()); Assert.assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get());
return completionService.submit(() -> { try { preExecute(); producer.produce(queue); } catch (Exception e) { logger.error("error consuming records", e); queue.markAsFailed(e); throw e; } finally { if (latch.getCount() == 0) { queue.close();
/** * Setup log and parquet reading in parallel. Both write to central buffer. */ @SuppressWarnings("unchecked") private List<BoundedInMemoryQueueProducer<ArrayWritable>> getParallelProducers() { List<BoundedInMemoryQueueProducer<ArrayWritable>> producers = new ArrayList<>(); producers.add(new FunctionBasedQueueProducer<>(buffer -> { logRecordScanner.scan(); return null; })); producers.add(new IteratorBasedQueueProducer<>(parquetRecordsIterator)); return producers; }
@Override protected List<WriteStatus> computeNext() { // Executor service used for launching writer thread. BoundedInMemoryExecutor<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor = null; try { final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); bufferedIteratorExecutor = new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), 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 { if (null != bufferedIteratorExecutor) { bufferedIteratorExecutor.shutdownNow(); } } }
/** * Start only consumer */ private Future<E> startConsumer() { return consumer.map(consumer -> { return executorService.submit( () -> { logger.info("starting consumer thread"); preExecute(); try { E result = consumer.consume(queue); logger.info("Queue Consumption is done; notifying producer threads"); return result; } catch (Exception e) { logger.error("error consuming records", e); queue.markAsFailed(e); throw e; } }); }).orElse(CompletableFuture.completedFuture(null)); }
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(); }
@Override public void produce(BoundedInMemoryQueue<I, ?> queue) throws Exception { logger.info("starting to buffer records"); while (inputIterator.hasNext()) { queue.insertRecord(inputIterator.next()); } logger.info("finished buffering records"); } }
/** * Main API to run both production and consumption */ public E execute() { try { ExecutorCompletionService<Boolean> producerService = startProducers(); Future<E> future = startConsumer(); // Wait for consumer to be done return future.get(); } catch (Exception e) { throw new HoodieException(e); } }
@Override public void close() throws IOException { this.parquetRecordsIterator.close(); this.executor.shutdownNow(); }
public BoundedInMemoryExecutor(final long bufferLimitInBytes, List<BoundedInMemoryQueueProducer<I>> producers, Optional<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction, final SizeEstimator<O> sizeEstimator) { this.producers = producers; this.consumer = consumer; // Ensure single thread for each producer thread and one for consumer this.executorService = Executors.newFixedThreadPool(producers.size() + 1); this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); }
/** * Inserts record into queue after applying transformation * * @param t Item to be queueed */ public void insertRecord(I t) throws Exception { // If already closed, throw exception if (isWriteDone.get()) { throw new IllegalStateException("Queue closed for enqueueing new entries"); } // We need to stop queueing if queue-reader has failed and exited. throwExceptionIfFailed(); rateLimiter.acquire(); // We are retrieving insert value in the record queueing thread to offload computation // around schema validation // and record creation to it. final O payload = transformFunction.apply(t); adjustBufferSizeIfNeeded(payload); queue.put(Optional.of(payload)); }
/** * Reader interface but never exposed to outside world as this is a single consumer queue. * Reading is done through a singleton iterator for this queue. */ private Optional<O> readNextRecord() { if (this.isReadDone.get()) { return Optional.empty(); } rateLimiter.release(); Optional<O> newRecord = Optional.empty(); while (expectMoreRecords()) { try { throwExceptionIfFailed(); newRecord = queue.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS); if (newRecord != null) { break; } } catch (InterruptedException e) { logger.error("error reading records from queue", e); throw new HoodieException(e); } } if (newRecord != null && newRecord.isPresent()) { return newRecord; } else { // We are done reading all the records from internal iterator. this.isReadDone.set(true); return Optional.empty(); } }
@Override protected List<WriteStatus> computeNext() { // Executor service used for launching writer thread. BoundedInMemoryExecutor<HoodieRecord<T>, Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> bufferedIteratorExecutor = null; try { final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); bufferedIteratorExecutor = new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), 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 { if (null != bufferedIteratorExecutor) { bufferedIteratorExecutor.shutdownNow(); } } }
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(); }