himanshug commented on a change in pull request #8578: parallel broker merges on fork join pool URL: https://github.com/apache/incubator-druid/pull/8578#discussion_r334212480
########## File path: core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java ########## @@ -0,0 +1,1077 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.java.util.common.guava; + +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.JvmUtils; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.RecursiveAction; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BinaryOperator; + +/** + * Artisanal, locally-sourced, hand-crafted, gluten and GMO free, bespoke, small-batch parallel merge combinining sequence + */ +public class ParallelMergeCombiningSequence<T> extends YieldingSequenceBase<T> +{ + private static final Logger LOG = new Logger(ParallelMergeCombiningSequence.class); + + private final ForkJoinPool workerPool; + private final List<Sequence<T>> baseSequences; + private final Ordering<T> orderingFn; + private final BinaryOperator<T> combineFn; + private final int queueSize; + private final boolean hasTimeout; + private final long timeoutAtNanos; + private final int queryPriority; // not currently used :( + private final int yieldAfter; + private final int batchSize; + private final int parallelism; + private final CancellationGizmo cancellationGizmo; + + public ParallelMergeCombiningSequence( + ForkJoinPool workerPool, + List<Sequence<T>> baseSequences, + Ordering<T> orderingFn, + BinaryOperator<T> combineFn, + boolean hasTimeout, + long timeoutMillis, + int queryPriority, + int parallelism, + int yieldAfter, + int batchSize + ) + { + this.workerPool = workerPool; + this.baseSequences = baseSequences; + this.orderingFn = orderingFn; + this.combineFn = combineFn; + this.hasTimeout = hasTimeout; + this.timeoutAtNanos = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutMillis, TimeUnit.MILLISECONDS); + this.queryPriority = queryPriority; + this.parallelism = parallelism; + this.yieldAfter = yieldAfter; + this.batchSize = batchSize; + this.queueSize = 4 * (yieldAfter / batchSize); + this.cancellationGizmo = new CancellationGizmo(); + } + + @Override + public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator) + { + if (baseSequences.isEmpty()) { + return Sequences.<T>empty().toYielder(initValue, accumulator); + } + + final BlockingQueue<ResultBatch<T>> outputQueue = new ArrayBlockingQueue<>(queueSize); + MergeCombinePartitioningAction<T> finalMergeAction = new MergeCombinePartitioningAction<>( + baseSequences, + orderingFn, + combineFn, + outputQueue, + queueSize, + parallelism, + yieldAfter, + batchSize, + hasTimeout, + timeoutAtNanos, + cancellationGizmo + ); + workerPool.execute(finalMergeAction); + Sequence<T> finalOutSequence = makeOutputSequenceForQueue(outputQueue, hasTimeout, timeoutAtNanos, cancellationGizmo); + return finalOutSequence.toYielder(initValue, accumulator); + } + + /** + * Create an output {@link Sequence} that wraps the output {@link BlockingQueue} of a + * {@link MergeCombinePartitioningAction} + */ + static <T> Sequence<T> makeOutputSequenceForQueue( + BlockingQueue<ResultBatch<T>> queue, + boolean hasTimeout, + long timeoutAtNanos, + CancellationGizmo cancellationGizmo + ) + { + return new BaseSequence<>( + new BaseSequence.IteratorMaker<T, Iterator<T>>() + { + @Override + public Iterator<T> make() + { + return new Iterator<T>() + { + private ResultBatch<T> currentBatch; + + @Override + public boolean hasNext() + { + final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); + if (thisTimeoutNanos < 0) { + throw new RE(new TimeoutException("Sequence iterator timed out")); + } + + if (currentBatch != null && !currentBatch.isTerminalResult() && !currentBatch.isDrained()) { + return true; + } + try { + if (currentBatch == null || currentBatch.isDrained()) { + if (hasTimeout) { + currentBatch = queue.poll(thisTimeoutNanos, TimeUnit.NANOSECONDS); + } else { + currentBatch = queue.take(); + } + } + if (currentBatch == null) { + throw new RE(new TimeoutException("Sequence iterator timed out waiting for data")); + } + + if (cancellationGizmo.isCancelled()) { + throw cancellationGizmo.getRuntimeException(); + } + + if (currentBatch.isTerminalResult()) { + return false; + } + return true; + } + catch (InterruptedException e) { + throw new RE(e); + } + } + + @Override + public T next() + { + if (cancellationGizmo.isCancelled()) { + throw cancellationGizmo.getRuntimeException(); + } + + if (currentBatch == null || currentBatch.isDrained() || currentBatch.isTerminalResult()) { + throw new NoSuchElementException(); + } + return currentBatch.next(); + } + }; + } + + @Override + public void cleanup(Iterator<T> iterFromMake) + { + // nothing to cleanup + } + } + ); + } + + /** + * This {@link RecursiveAction} is the initial task of the parallel merge-combine process. Capacity and input sequence + * count permitting, it will partition the input set of {@link Sequence} to do 2 layer parallel merge. + * + * For the first layer, the partitions of input sequences are each wrapped in {@link YielderBatchedResultsCursor}, and + * for each partition a {@link PrepareMergeCombineInputsAction} will be executed to to wait for each of the yielders to + * yield {@link ResultBatch}. After the cursors all have an initial set of results, the + * {@link PrepareMergeCombineInputsAction} will execute a {@link MergeCombineAction} + * to perform the actual work of merging sequences and combining results. The merged and combined output of each + * partition will itself be put into {@link ResultBatch} and pushed to a {@link BlockingQueue} with a + * {@link ForkJoinPool} {@link QueuePusher}. + * + * The second layer will execute a single {@link PrepareMergeCombineInputsAction} to wait for the {@link ResultBatch} + * from each partition to be available in their 'output' {@link BlockingQueue} which each is wrapped in + * {@link BlockingQueueuBatchedResultsCursor}. Like the first layer, after the {@link PrepareMergeCombineInputsAction} + * is complete and some {@link ResultBatch} are ready to merge from each partition, it will execute a + * {@link MergeCombineAction} do a final merge combine of all the parallel computed results, again pushing + * {@link ResultBatch} into a {@link BlockingQueue} with a {@link QueuePusher}. + */ + private static class MergeCombinePartitioningAction<T> extends RecursiveAction + { + private final List<Sequence<T>> sequences; + private final Ordering<T> orderingFn; + private final BinaryOperator<T> combineFn; + private final BlockingQueue<ResultBatch<T>> out; + private final int queueSize; + private final int parallelism; + private final int yieldAfter; + private final int batchSize; + private final boolean hasTimeout; + private final long timeoutAt; + private final CancellationGizmo cancellationGizmo; + + private MergeCombinePartitioningAction( + List<Sequence<T>> sequences, + Ordering<T> orderingFn, + BinaryOperator<T> combineFn, + BlockingQueue<ResultBatch<T>> out, + int queueSize, + int parallelism, + int yieldAfter, + int batchSize, + boolean hasTimeout, + long timeoutAt, + CancellationGizmo cancellationGizmo + ) + { + this.sequences = sequences; + this.combineFn = combineFn; + this.orderingFn = orderingFn; + this.out = out; + this.queueSize = queueSize; + this.parallelism = parallelism; + this.yieldAfter = yieldAfter; + this.batchSize = batchSize; + this.hasTimeout = hasTimeout; + this.timeoutAt = timeoutAt; + this.cancellationGizmo = cancellationGizmo; + } + + @Override + protected void compute() + { + try { + final int parallelTaskCount = computeNumTasks(); Review comment: what if multiple queries arrive concurrently, there would be races and they might overestimate available parallel capacity ? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org For additional commands, e-mail: commits-h...@druid.apache.org