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

228 wiersze
8.0 KiB
Java
Czysty Zwykły widok Historia

package com.onthegomap.planetiler.worker;
2021-04-10 09:25:42 +00:00
2022-02-24 01:32:41 +00:00
import com.onthegomap.planetiler.collection.IterableOnce;
import com.onthegomap.planetiler.stats.Counter;
import com.onthegomap.planetiler.stats.Stats;
2021-04-16 00:54:33 +00:00
import java.util.ArrayDeque;
2021-07-27 12:09:06 +00:00
import java.util.List;
2021-04-16 00:54:33 +00:00
import java.util.Queue;
import java.util.concurrent.ArrayBlockingQueue;
2021-09-10 00:46:20 +00:00
import java.util.concurrent.BlockingDeque;
2021-04-16 00:54:33 +00:00
import java.util.concurrent.BlockingQueue;
2021-08-06 01:10:01 +00:00
import java.util.concurrent.CopyOnWriteArrayList;
2021-04-16 00:54:33 +00:00
import java.util.concurrent.TimeUnit;
2021-07-27 12:09:06 +00:00
import java.util.concurrent.atomic.AtomicReference;
2021-04-10 09:25:42 +00:00
import java.util.function.Consumer;
2021-09-10 00:46:20 +00:00
/**
* A high-performance blocking queue to hand off work from producing threads to consuming threads.
* <p>
* Wraps a standard {@link BlockingDeque}, with a few customizations:
* <ul>
* <li>items are buffered into configurable-sized batches before putting on the actual queue to reduce contention</li>
* <li>writers can mark the queue "finished" with {@link #close()} and readers will get {@code null} when there are no
* more items to read</li>
2021-09-10 00:46:20 +00:00
* </ul>
* <p>
* Once a thread starts reading from this queue, it needs to finish otherwise all items might not be read.
*
* @param <T> the type of elements held in this queue
*/
2022-02-24 01:32:41 +00:00
public class WorkQueue<T> implements AutoCloseable, IterableOnce<T>, Consumer<T> {
2021-04-10 09:25:42 +00:00
2021-04-16 00:54:33 +00:00
private final BlockingQueue<Queue<T>> itemQueue;
private final int batchSize;
2021-08-06 01:10:01 +00:00
private final List<WriterForThread> writers = new CopyOnWriteArrayList<>();
2021-09-10 00:46:20 +00:00
private final ThreadLocal<WriterForThread> writerProvider = ThreadLocal.withInitial(WriterForThread::new);
2021-08-06 01:10:01 +00:00
private final List<ReaderForThread> readers = new CopyOnWriteArrayList<>();
2021-09-10 00:46:20 +00:00
private final ThreadLocal<ReaderForThread> readerProvider = ThreadLocal.withInitial(ReaderForThread::new);
2021-04-16 00:54:33 +00:00
private final int pendingBatchesCapacity;
2021-07-27 02:01:55 +00:00
private final Counter.MultiThreadCounter enqueueCountStatAll;
private final Counter.MultiThreadCounter enqueueBlockTimeNanosAll;
private final Counter.MultiThreadCounter dequeueCountStatAll;
private final Counter.MultiThreadCounter dequeueBlockTimeNanosAll;
private final Counter.MultiThreadCounter pendingCountAll = Counter.newMultiThreadCounter();
2021-09-10 00:46:20 +00:00
private volatile boolean hasIncomingData = true;
2021-04-11 20:10:28 +00:00
2021-09-10 00:46:20 +00:00
/**
* @param name ID to prepend to stats generated about this queue
* @param capacity maximum number of pending items that can be held in the queue
* @param maxBatch batch size to buffer elements into before handing off to the blocking queue
* @param stats stats to monitor this with
*/
2021-04-16 00:54:33 +00:00
public WorkQueue(String name, int capacity, int maxBatch, Stats stats) {
this.pendingBatchesCapacity = capacity / maxBatch;
this.batchSize = maxBatch;
itemQueue = new ArrayBlockingQueue<>(pendingBatchesCapacity);
2021-06-05 12:02:51 +00:00
stats.gauge(name + "_blocking_queue_capacity", () -> pendingBatchesCapacity);
stats.gauge(name + "_blocking_queue_size", itemQueue::size);
stats.gauge(name + "_capacity", this::getCapacity);
stats.gauge(name + "_size", this::getPending);
2021-07-27 02:01:55 +00:00
this.enqueueCountStatAll = stats.longCounter(name + "_enqueue_count");
this.enqueueBlockTimeNanosAll = stats.nanoCounter(name + "_enqueue_block_time_seconds");
this.dequeueCountStatAll = stats.longCounter(name + "_dequeue_count");
this.dequeueBlockTimeNanosAll = stats.nanoCounter(name + "_dequeue_block_time_seconds");
2021-04-11 20:10:28 +00:00
}
2021-04-10 09:25:42 +00:00
@Override
2021-04-11 20:10:28 +00:00
public void close() {
2021-04-16 11:13:05 +00:00
try {
2021-08-06 01:10:01 +00:00
for (var writer : writers) {
var q = writer.writeBatchRef.get();
if (q != null && !q.isEmpty()) {
itemQueue.put(q);
2021-04-16 00:54:33 +00:00
}
}
2021-08-06 01:10:01 +00:00
hasIncomingData = false;
2021-04-23 09:44:29 +00:00
} catch (Exception e) {
2021-04-16 11:13:05 +00:00
throw new RuntimeException(e);
2021-04-16 00:54:33 +00:00
}
2021-04-10 09:25:42 +00:00
}
2021-09-10 00:46:20 +00:00
/** Returns a writer optimized to accept items from a single thread. */
2021-07-27 02:01:55 +00:00
public Consumer<T> threadLocalWriter() {
return writerProvider.get();
}
2021-09-10 00:46:20 +00:00
/** Returns a reader optimized to produce items for a single thread. */
2022-02-24 01:32:41 +00:00
public IterableOnce<T> threadLocalReader() {
2021-07-27 02:01:55 +00:00
return readerProvider.get();
}
2021-04-10 09:25:42 +00:00
@Override
2021-04-16 00:54:33 +00:00
public void accept(T item) {
2021-07-27 02:01:55 +00:00
writerProvider.get().accept(item);
}
@Override
public T get() {
return readerProvider.get().get();
}
2021-09-10 00:46:20 +00:00
/**
* Returns the number of enqueued items that have not been dequeued yet.
* <p>
* NOTE: this may be larger than the initial capacity because each writer thread can buffer items into a batch and
* each reader thread might be reading items from a batch.
*/
public int getPending() {
return (int) pendingCountAll.get();
}
/**
* Returns the total number of items that can be pending.
* <p>
* This will be larger than the initial capacity because each writer thread can buffer items into a batch and each
* reader thread can read items from a batch.
*/
public int getCapacity() {
// actual queue can hold more than the specified capacity because each writer and reader may have a batch they are
// working on that is outside the queue
return (pendingBatchesCapacity + writers.size() + readers.size()) * batchSize;
}
/**
* Caches thread-local values so that a single thread can accept new items without having to do thread-local lookups.
*/
2021-07-27 02:01:55 +00:00
private class WriterForThread implements Consumer<T> {
2021-09-10 00:46:20 +00:00
final AtomicReference<Queue<T>> writeBatchRef = new AtomicReference<>(null);
2021-07-27 02:01:55 +00:00
Queue<T> writeBatch = null;
2021-09-10 00:46:20 +00:00
final Counter pendingCount = pendingCountAll.counterForThread();
final Counter enqueueCountStat = enqueueCountStatAll.counterForThread();
final Counter enqueueBlockTimeNanos = enqueueBlockTimeNanosAll.counterForThread();
2021-07-27 02:01:55 +00:00
2021-09-10 00:46:20 +00:00
private WriterForThread() {
2021-07-27 12:09:06 +00:00
writers.add(this);
}
2021-07-27 02:01:55 +00:00
@Override
public void accept(T item) {
// past 4-8 concurrent writers, start getting lock contention adding to the blocking queue so add to the
2021-08-06 01:10:01 +00:00
// queue in less frequent, larger batches
2021-07-27 02:01:55 +00:00
if (writeBatch == null) {
writeBatch = new ArrayDeque<>(batchSize);
2021-07-27 12:09:06 +00:00
writeBatchRef.set(writeBatch);
2021-07-27 02:01:55 +00:00
}
2021-04-16 00:54:33 +00:00
2021-07-27 02:01:55 +00:00
writeBatch.offer(item);
pendingCount.inc();
2021-04-16 00:54:33 +00:00
2021-07-27 02:01:55 +00:00
if (writeBatch.size() >= batchSize) {
flushWrites();
}
enqueueCountStat.inc();
2021-04-16 00:54:33 +00:00
}
2021-04-10 09:25:42 +00:00
2021-07-27 02:01:55 +00:00
private void flushWrites() {
if (writeBatch != null && !writeBatch.isEmpty()) {
try {
Queue<T> oldWriteBatch = writeBatch;
writeBatch = null;
2021-07-27 12:09:06 +00:00
writeBatchRef.set(null);
2021-07-27 02:01:55 +00:00
// blocks if full
if (!itemQueue.offer(oldWriteBatch)) {
long start = System.nanoTime();
itemQueue.put(oldWriteBatch);
enqueueBlockTimeNanos.incBy(System.nanoTime() - start);
}
} catch (InterruptedException ex) {
throw new RuntimeException(ex);
2021-04-16 00:54:33 +00:00
}
}
}
2021-04-10 09:25:42 +00:00
}
/**
* Caches thread-local values so that a single thread can read new items without having to do thread-local lookups.
*/
2022-02-24 01:32:41 +00:00
private class ReaderForThread implements IterableOnce<T> {
2021-04-16 00:54:33 +00:00
2021-07-27 02:01:55 +00:00
Queue<T> readBatch = null;
2021-09-10 00:46:20 +00:00
final Counter dequeueBlockTimeNanos = dequeueBlockTimeNanosAll.counterForThread();
final Counter pendingCount = pendingCountAll.counterForThread();
final Counter dequeueCountStat = dequeueCountStatAll.counterForThread();
2021-07-27 02:01:55 +00:00
2021-08-06 01:10:01 +00:00
ReaderForThread() {
readers.add(this);
}
2021-07-27 02:01:55 +00:00
@Override
public T get() {
Queue<T> itemBatch = readBatch;
2021-04-16 00:54:33 +00:00
2021-07-27 02:01:55 +00:00
if (itemBatch == null || itemBatch.isEmpty()) {
long start = System.nanoTime();
do {
if (!hasIncomingData && itemQueue.isEmpty()) {
break;
}
if ((itemBatch = itemQueue.poll()) == null) {
try {
itemBatch = itemQueue.poll(100, TimeUnit.MILLISECONDS);
if (itemBatch != null) {
break;
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
break;// signal EOF
2021-04-16 00:54:33 +00:00
}
}
2021-07-27 02:01:55 +00:00
} while (itemBatch == null);
readBatch = itemBatch;
dequeueBlockTimeNanos.incBy(System.nanoTime() - start);
}
2021-04-16 00:54:33 +00:00
2021-07-27 02:01:55 +00:00
T result = itemBatch == null ? null : itemBatch.poll();
if (result != null) {
pendingCount.incBy(-1);
}
dequeueCountStat.inc();
return result;
2021-04-16 00:54:33 +00:00
}
}
2021-04-10 09:25:42 +00:00
}