planetiler/planetiler-core/src/main/java/com/onthegomap/planetiler/worker/WorkerPipeline.java

289 wiersze
10 KiB
Java
Czysty Zwykły widok Historia

package com.onthegomap.planetiler.worker;
2021-04-11 20:10:28 +00:00
import static com.onthegomap.planetiler.worker.Worker.joinFutures;
2021-08-05 11:02:35 +00:00
2022-02-24 01:32:41 +00:00
import com.onthegomap.planetiler.collection.IterableOnce;
import com.onthegomap.planetiler.stats.ProgressLoggers;
import com.onthegomap.planetiler.stats.Stats;
2021-04-14 11:20:19 +00:00
import java.time.Duration;
import java.util.Collection;
2021-04-12 10:54:52 +00:00
import java.util.Iterator;
2021-08-05 11:02:35 +00:00
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
2021-04-11 20:10:28 +00:00
import java.util.function.Consumer;
2021-09-10 00:46:20 +00:00
/**
* A mini-framework for chaining sequential steps that run in dedicated threads with a queue between each.
* <p>
* For example:
*
* <pre>
* {@code
2021-09-10 00:46:20 +00:00
* WorkerPipeline.start("name", stats)
* .readFrom("reader", List.of(1, 2, 3))
* .addBuffer("reader_queue", 10)
* .addWorker("process", 2, (i, next) -> next.accept(doExpensiveWork(i))
* .addBuffer("writer_queue", 10)
* .sinkToConsumer("writer", 1, result -> writeToDisk(result))
* .await();
* }
* </pre>
2021-09-10 00:46:20 +00:00
* <p>
* NOTE: to do any forking/joining, you must construct and wire-up queues and each sequence of steps manually.
*
* @param <T> input type of this pipeline
*/
public record WorkerPipeline<T> (
2021-04-11 20:10:28 +00:00
String name,
2021-08-05 11:09:52 +00:00
WorkerPipeline<?> previous,
2021-04-11 20:10:28 +00:00
WorkQueue<T> inputQueue,
2021-08-05 11:02:35 +00:00
Worker worker,
CompletableFuture<?> done
2021-04-11 20:10:28 +00:00
) {
2021-09-10 00:46:20 +00:00
/*
* Empty/Bufferable/Builder are used to provide a fluent API for building a model of the steps to run (and keep
* pointers to workers and queues) and Builder.build converts to the top-level WorkerPipeline that clients
* can use to wait on results.
*/
2021-04-11 20:10:28 +00:00
2021-09-10 00:46:20 +00:00
/** Returns a new pipeline builder where all worker and queue names will start with {@code prefix_}. */
2021-04-12 11:14:05 +00:00
public static Empty start(String prefix, Stats stats) {
return new Empty(prefix, stats);
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Blocks until all work has been completed by all steps of this pipeline, logging progress at a fixed {@code
* logInterval}.
*
* @throws RuntimeException if interrupted or if any of the threads die with an exception.
*/
2021-04-16 11:30:33 +00:00
public void awaitAndLog(ProgressLoggers loggers, Duration logInterval) {
2021-08-05 11:02:35 +00:00
loggers.awaitAndLog(done, logInterval);
2021-04-16 11:30:33 +00:00
loggers.log();
}
2021-09-10 00:46:20 +00:00
/**
* Blocks until all work has been completed by all steps of this pipeline.
*
* @throws RuntimeException if interrupted or if any of the threads die with an exception.
*/
2021-04-16 11:13:05 +00:00
public void await() {
2021-08-05 11:02:35 +00:00
try {
done.get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
2021-04-16 11:13:05 +00:00
}
}
2021-09-10 00:46:20 +00:00
/**
* Work that happens in a thread at the start of a pipeline to provide the initial elements to process.
*
* @param <O> type of items that this step will emit
*/
@FunctionalInterface
2021-04-11 20:10:28 +00:00
public interface SourceStep<O> {
2021-09-10 00:46:20 +00:00
/**
* Called inside each worker thread to emit elements until there are no more, then return.
*
* @param next call {@code next.accept} to pass elements to the next step of the pipeline
* @throws Exception if an error occurs, will be rethrown by {@link #await()} as a {@link RuntimeException}
*/
2021-04-12 10:05:32 +00:00
void run(Consumer<O> next) throws Exception;
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Work that happens in a thread in the middle of a pipeline that accepts elements from previous steps and emits
* elements to the next step.
*
* @param <I> type of items that this step consumes
* @param <O> type of items that this step emits
*/
@FunctionalInterface
2021-04-11 20:10:28 +00:00
public interface WorkerStep<I, O> {
2021-09-10 00:46:20 +00:00
/**
* Called inside each worker thread to process elements until there are no more, then return.
*
* @param prev get elements from the previous step using {@code prev.get}, will return null when there are no more
* elements to process
* @param next call {@code next.accept} to pass items to the next step of the pipeline
* @throws Exception if an error occurs, will be rethrown by {@link #await()} as a {@link RuntimeException}
*/
2022-02-24 01:32:41 +00:00
void run(IterableOnce<I> prev, Consumer<O> next) throws Exception;
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Work that happens in a thread at the end of a pipeline that accepts elements from the previous step.
*
* @param <I> type of items that this step consumes
*/
@FunctionalInterface
2021-04-11 20:10:28 +00:00
public interface SinkStep<I> {
2021-09-10 00:46:20 +00:00
/**
* Called inside each worker thread to consume elements until there are no more, then return.
*
* @param prev get elements from the previous step using {@code prev.get}, will return null when there are no more
* elements to process
* @throws Exception if an error occurs, will be rethrown by {@link #await()} as a {@link RuntimeException}
*/
2022-02-24 01:32:41 +00:00
void run(IterableOnce<I> prev) throws Exception;
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* An intermediate step while building a pipeline that requires the user to add a queue to buffer items before adding
* the next step.
*
* @param <E> type of elements coming out of the previous step that need to be stored in a queue
*/
public interface Bufferable<E> {
2021-04-11 20:10:28 +00:00
2021-09-10 00:46:20 +00:00
/**
* Adds a {@link WorkQueue} that groups items into batches before enqueueing them to reduce contention when many
* threads are reading or writing to the queue simultaneously.
*/
Builder<E> addBuffer(String name, int size, int batchSize);
2021-04-11 20:10:28 +00:00
2021-09-10 00:46:20 +00:00
/**
* Adds a {@link WorkQueue} with batching disabled.
*/
default Builder<E> addBuffer(String name, int size) {
2021-04-11 20:10:28 +00:00
return addBuffer(name, size, 1);
}
}
2021-09-10 00:46:20 +00:00
/** Builder for a new topology that does not yet have any steps. */
2022-02-24 01:32:41 +00:00
public record Empty(String prefix, Stats stats) {
2021-04-12 11:14:05 +00:00
2021-09-10 00:46:20 +00:00
/**
* Adds an initial step that runs {@code producer} in {@code threads} worker threads to produce items for this
* queue.
*/
public <T> Bufferable<T> fromGenerator(String name, SourceStep<T> producer, int threads) {
2021-04-12 11:14:05 +00:00
return (queueName, size, batchSize) -> {
2021-04-16 00:54:33 +00:00
var nextQueue = new WorkQueue<T>(prefix + "_" + queueName, size, batchSize, stats);
2021-07-27 02:01:55 +00:00
Worker worker = new Worker(prefix + "_" + name, stats, threads,
() -> producer.run(nextQueue.threadLocalWriter()));
2021-04-12 11:14:05 +00:00
return new Builder<>(prefix, name, nextQueue, worker, stats);
};
}
2021-09-10 00:46:20 +00:00
/**
* Adds an initial step that runs {@code producer} in 1 worker thread to produce items for this queue.
*/
public <T> Bufferable<T> fromGenerator(String name, SourceStep<T> producer) {
2021-04-12 11:14:05 +00:00
return fromGenerator(name, producer, 1);
}
2021-09-10 00:46:20 +00:00
/**
* Adds an initial step that reads all items from {@code iterable} in a single worker thread to produce items for
* this queue.
*/
public <T> Bufferable<T> readFrom(String name, Iterable<T> iterable) {
2021-04-25 21:08:01 +00:00
Iterator<T> iter = iterable.iterator();
2021-04-12 11:14:05 +00:00
return fromGenerator(name, next -> {
while (iter.hasNext()) {
next.accept(iter.next());
}
}, 1);
}
2021-09-10 00:46:20 +00:00
/**
* Populates an initial queue with {@code items} for subsequent steps to process but does not kick off a worker
* thread.
*/
public <T> Builder<T> readFromTiny(String name, Collection<T> items) {
WorkQueue<T> queue = new WorkQueue<>(prefix + "_" + name, items.size(), 1, stats);
2021-07-27 02:01:55 +00:00
Consumer<T> writer = queue.threadLocalWriter();
for (T item : items) {
2021-07-27 02:01:55 +00:00
writer.accept(item);
}
2021-08-05 01:22:20 +00:00
queue.close();
return readFromQueue(queue);
}
2021-09-10 00:46:20 +00:00
/**
* Starts the pipeline from a queue that is populated externally, and does not kick off any threads.
*/
public <T> Builder<T> readFromQueue(WorkQueue<T> input) {
2021-07-24 00:20:51 +00:00
return new Builder<>(prefix, input, stats);
2021-04-12 11:14:05 +00:00
}
}
2021-09-10 00:46:20 +00:00
/**
* A step while building a pipeline that needs a subsequent step to process elements.
*
* @param <O> type of elements that the next step must process
*/
public record Builder<O> (
2021-04-12 11:14:05 +00:00
String prefix,
2021-04-11 20:10:28 +00:00
String name,
2021-09-10 00:46:20 +00:00
// keep track of previous elements so that build can wire-up the computation graph
WorkerPipeline.Builder<?> previous,
WorkQueue<?> inputQueue,
2021-04-11 20:10:28 +00:00
WorkQueue<O> outputQueue,
2021-09-10 00:46:20 +00:00
Worker worker,
Stats stats
2021-04-11 20:10:28 +00:00
) {
2021-09-10 00:46:20 +00:00
private Builder(String prefix, String name, WorkQueue<O> outputQueue, Worker worker, Stats stats) {
2021-04-12 11:14:05 +00:00
this(prefix, name, null, null, outputQueue, worker, stats);
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
private Builder(String prefix, WorkQueue<O> outputQueue, Stats stats) {
2021-07-24 00:20:51 +00:00
this(prefix, null, outputQueue, null, stats);
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Runs {@code step} simultaneously in {@code threads} threads which consumes items and emits new ones that must be
* buffered.
*
* @param <O2> type of element that this step emits
*/
public <O2> Bufferable<O2> addWorker(String name, int threads, WorkerStep<O, O2> step) {
Builder<O> curr = this;
2021-04-11 20:10:28 +00:00
return (queueName, size, batchSize) -> {
2021-04-16 00:54:33 +00:00
var nextOutputQueue = new WorkQueue<O2>(prefix + "_" + queueName, size, batchSize, stats);
2021-07-27 02:01:55 +00:00
var worker = new Worker(prefix + "_" + name, stats, threads,
() -> step.run(outputQueue.threadLocalReader(), nextOutputQueue.threadLocalWriter()));
2021-04-12 11:14:05 +00:00
return new Builder<>(prefix, name, curr, outputQueue, nextOutputQueue, worker, stats);
2021-04-11 20:10:28 +00:00
};
}
2021-09-10 00:46:20 +00:00
private WorkerPipeline<?> build() {
2021-08-05 11:09:52 +00:00
var previousPipeline = previous == null || previous.worker == null ? null : previous.build();
2021-08-05 11:02:35 +00:00
var doneFuture = worker != null ? worker.done() : CompletableFuture.completedFuture(true);
2021-08-05 11:09:52 +00:00
if (previousPipeline != null) {
doneFuture = joinFutures(doneFuture, previousPipeline.done);
2021-08-05 11:02:35 +00:00
}
2021-08-05 11:58:13 +00:00
if (worker != null && outputQueue != null) {
2021-08-05 11:02:35 +00:00
doneFuture = doneFuture.thenRun(outputQueue::close);
}
2021-08-05 11:09:52 +00:00
return new WorkerPipeline<>(name, previousPipeline, inputQueue, worker, doneFuture);
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Runs {@code step} simultaneously in {@code threads} threads that consumes items but does not emit any.
*/
2021-08-05 11:09:52 +00:00
public WorkerPipeline<O> sinkTo(String name, int threads, SinkStep<O> step) {
var previousPipeline = build();
2021-07-27 02:01:55 +00:00
var worker = new Worker(prefix + "_" + name, stats, threads, () -> step.run(outputQueue.threadLocalReader()));
2021-08-05 11:09:52 +00:00
var doneFuture = joinFutures(worker.done(), previousPipeline.done);
return new WorkerPipeline<>(name, previousPipeline, outputQueue, worker, doneFuture);
2021-04-11 20:10:28 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Runs {@code threads} simultaneous worker threads that consume items from previous step and invoke {@code
* consumer.accept} for each one.
*/
public WorkerPipeline<O> sinkToConsumer(String name, int threads, Consumer<O> consumer) {
2021-04-11 20:10:28 +00:00
return sinkTo(name, threads, (prev) -> {
2022-02-24 01:32:41 +00:00
for (O item : prev) {
2021-09-10 00:46:20 +00:00
consumer.accept(item);
2021-04-11 20:10:28 +00:00
}
});
}
}
}