1
0

[HUDI-2439] Remove SparkBoundedInMemoryExecutor (#4860)

This commit is contained in:
Raymond Xu
2022-02-26 05:02:12 -08:00
committed by GitHub
parent 1379300b5b
commit c77b2591d0
14 changed files with 99 additions and 113 deletions

View File

@@ -25,6 +25,11 @@ import java.io.Serializable;
*/
public interface Functions {
static Runnable noop() {
return () -> {
};
}
/**
* A function which has not any parameter.
*/

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.util.queue;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Functions;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.exception.HoodieException;
@@ -26,7 +27,8 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
@@ -54,29 +56,35 @@ public class BoundedInMemoryExecutor<I, O, E> {
private final List<BoundedInMemoryQueueProducer<I>> producers;
// Consumer
private final Option<BoundedInMemoryQueueConsumer<O, E>> consumer;
// pre-execute function to implement environment specific behavior before executors (producers/consumer) run
private final Runnable preExecuteRunnable;
public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator<I> inputItr,
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> transformFunction, Runnable preExecuteRunnable) {
this(bufferLimitInBytes, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, preExecuteRunnable);
}
public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer<I> producer,
Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction) {
this(bufferLimitInBytes, Arrays.asList(producer), consumer, transformFunction, new DefaultSizeEstimator<>());
this(bufferLimitInBytes, producer, consumer, transformFunction, Functions.noop());
}
public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer<I> producer,
Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction, Runnable preExecuteRunnable) {
this(bufferLimitInBytes, Collections.singletonList(producer), consumer, transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable);
}
public BoundedInMemoryExecutor(final long bufferLimitInBytes, List<BoundedInMemoryQueueProducer<I>> producers,
Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction,
final SizeEstimator<O> sizeEstimator) {
final SizeEstimator<O> sizeEstimator, Runnable preExecuteRunnable) {
this.producers = producers;
this.consumer = consumer;
this.preExecuteRunnable = preExecuteRunnable;
// 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);
}
/**
* Callback to implement environment specific behavior before executors (producers/consumer) run.
*/
public void preExecute() {
// Do Nothing in general context
}
/**
* Start all Producers.
*/
@@ -88,7 +96,7 @@ public class BoundedInMemoryExecutor<I, O, E> {
producers.stream().map(producer -> {
return completionService.submit(() -> {
try {
preExecute();
preExecuteRunnable.run();
producer.produce(queue);
} catch (Throwable e) {
LOG.error("error producing records", e);
@@ -116,7 +124,7 @@ public class BoundedInMemoryExecutor<I, O, E> {
return consumer.map(consumer -> {
return executorService.submit(() -> {
LOG.info("starting consumer thread");
preExecute();
preExecuteRunnable.run();
try {
E result = consumer.consume(queue);
LOG.info("Queue Consumption is done; notifying producer threads");