From f72501a3b27194ef99d1c7d5cdccf20a991d9e28 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 3 Sep 2019 02:30:28 -0700 Subject: [PATCH 01/51] sketch of broker parallel merges done in small batches on fork join pool --- .../CachingClusteredClientBenchmark.java | 70 +- .../druid/common/guava/CombiningSequence.java | 2 +- .../guava/ParallelMergeCombiningSequence.java | 882 ++++++++++++++++++ .../common/guava/ComplexSequenceTest.java | 2 +- .../ParallelMergeCombiningSequenceTest.java | 236 +++++ .../movingaverage/MovingAverageQueryTest.java | 5 +- .../druid/guice/LifecycleForkJoinPool.java | 61 ++ .../druid/query/DruidProcessingConfig.java | 23 + .../org/apache/druid/query/QueryContexts.java | 29 + .../druid/client/CachingClusteredClient.java | 32 +- .../druid/guice/DruidProcessingModule.java | 14 + ...chingClusteredClientFunctionalityTest.java | 5 +- .../client/CachingClusteredClientTest.java | 11 +- 13 files changed, 1352 insertions(+), 20 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java create mode 100644 core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java create mode 100644 processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 3ddd8245f8ed..ba6ead5e51b8 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -47,7 +47,7 @@ import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.StupidPool; -import org.apache.druid.data.input.Row; +import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.concurrent.Execs; @@ -65,6 +65,7 @@ import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.FluentQueryRunnerBuilder; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; @@ -83,6 +84,7 @@ import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; @@ -132,12 +134,13 @@ import java.util.Map.Entry; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; @State(Scope.Benchmark) @Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") -@Warmup(iterations = 15) -@Measurement(iterations = 30) +@Warmup(iterations = 3) +@Measurement(iterations = 5) public class CachingClusteredClientBenchmark { private static final Logger LOG = new Logger(CachingClusteredClientBenchmark.class); @@ -145,22 +148,30 @@ public class CachingClusteredClientBenchmark private static final String DATA_SOURCE = "ds"; public static final ObjectMapper JSON_MAPPER; - @Param({"8"}) + @Param({"8", "24"}) private int numServers; - @Param({"4", "2", "1"}) - private int numProcessingThreads; + + @Param({"5"}) + private int parallelism; + + @Param({"parallel", "serial"}) + private String mergeCombineStyle; @Param({"75000"}) private int rowsPerSegment; - @Param({"all"}) + @Param({"all", "minute"}) private String queryGranularity; private QueryToolChestWarehouse toolChestWarehouse; private QueryRunnerFactoryConglomerate conglomerate; private CachingClusteredClient cachingClusteredClient; private ExecutorService processingPool; + private LifecycleForkJoinPool forkJoinPool; + + private boolean parallelCombine; + private boolean possiblyParallel; private Query query; @@ -171,6 +182,8 @@ public class CachingClusteredClientBenchmark Collections.singletonList(basicSchema.getDataInterval()) ); + private final int numProcessingThreads = 4; + static { JSON_MAPPER = new DefaultObjectMapper(); JSON_MAPPER.setInjectableValues( @@ -186,6 +199,16 @@ public void setup() { final String schemaName = "basic"; + switch (mergeCombineStyle) { + case "parallel": + parallelCombine = true; + break; + case "serial": + default: + parallelCombine = false; + break; + } + BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); Map queryableIndexes = new HashMap<>(numServers); @@ -295,6 +318,7 @@ public > QueryToolChest getToolChest } processingPool = Execs.multiThreaded(processingConfig.getNumThreads(), "caching-clustered-client-benchmark"); + forkJoinPool = new LifecycleForkJoinPool((int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true, 1000L); cachingClusteredClient = new CachingClusteredClient( toolChestWarehouse, serverView, @@ -302,7 +326,8 @@ public > QueryToolChest getToolChest JSON_MAPPER, new ForegroundCachePopulator(JSON_MAPPER, new CachePopulatorStats(), 0), new CacheConfig(), - new DruidHttpClientConfig() + new DruidHttpClientConfig(), + forkJoinPool ); } @@ -356,6 +381,7 @@ public void tearDown() throws IOException { closer.close(); processingPool.shutdown(); + forkJoinPool.stop(); } @Benchmark @@ -368,6 +394,12 @@ public void timeseriesQuery(Blackhole blackhole) .intervals(basicSchemaIntervalSpec) .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) .granularity(Granularity.fromString(queryGranularity)) + .context( + ImmutableMap.of( + QueryContexts.BROKER_PARALLEL_MERGE_KEY, parallelCombine, + QueryContexts.BROKER_PARALLELISM, parallelism + ) + ) .build(); final List> results = runQuery(); @@ -385,11 +417,17 @@ public void topNQuery(Blackhole blackhole) query = new TopNQueryBuilder() .dataSource(DATA_SOURCE) .intervals(basicSchemaIntervalSpec) - .dimension(new DefaultDimensionSpec("dimUniform", null)) + .dimension(new DefaultDimensionSpec("dimSequential", null)) .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) .granularity(Granularity.fromString(queryGranularity)) .metric("sumLongSequential") .threshold(10_000) // we are primarily measuring 'broker' merge time, so collect a significant number of results + .context( + ImmutableMap.of( + QueryContexts.BROKER_PARALLEL_MERGE_KEY, parallelCombine, + QueryContexts.BROKER_PARALLELISM, parallelism + ) + ) .build(); final List> results = runQuery(); @@ -409,16 +447,22 @@ public void groupByQuery(Blackhole blackhole) .setDataSource(DATA_SOURCE) .setQuerySegmentSpec(basicSchemaIntervalSpec) .setDimensions( - new DefaultDimensionSpec("dimUniform", null), - new DefaultDimensionSpec("dimZipf", null) + new DefaultDimensionSpec("dimSequentialHalfNull", null), + new DefaultDimensionSpec("dimSequential", null) ) .setAggregatorSpecs(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) .setGranularity(Granularity.fromString(queryGranularity)) + .setContext( + ImmutableMap.of( + QueryContexts.BROKER_PARALLEL_MERGE_KEY, parallelCombine, + QueryContexts.BROKER_PARALLELISM, parallelism + ) + ) .build(); - final List results = runQuery(); + final List results = runQuery(); - for (Row result : results) { + for (ResultRow result : results) { blackhole.consume(result); } } diff --git a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java index 9e9a7d77df57..bc4119b2b5b2 100644 --- a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java +++ b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java @@ -227,7 +227,7 @@ boolean accumulatedSomething() } } - private class CombiningAccumulator implements Accumulator + class CombiningAccumulator implements Accumulator { private OutType retVal; private final Accumulator accumulator; diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java new file mode 100644 index 000000000000..83cb23dc6191 --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -0,0 +1,882 @@ +/* + * 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 com.google.common.primitives.Ints; +import org.apache.druid.java.util.common.logger.Logger; + +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.function.BinaryOperator; +import java.util.stream.Collectors; + +/** + * Artisanal, locally-sourced, hand-crafted, gluten and GMO free, bespoke, small-batch parallel merge combinining sequence + */ +public class ParallelMergeCombiningSequence extends YieldingSequenceBase +{ + private static final Logger LOG = new Logger(ParallelMergeCombiningSequence.class); + + private final ForkJoinPool workerPool; + private final List> baseSequences; + private final Ordering orderingFn; + private final BinaryOperator combineFn; + private final int queueSize; + private final boolean hasTimeout; + private final long timeoutAt; + private final int queryPriority; // not currently used :( + private final int yieldAfter; + private final int batchSize; + private final int parallelism; + + public ParallelMergeCombiningSequence( + ForkJoinPool workerPool, + List> baseSequences, + Ordering orderingFn, + BinaryOperator combineFn, + int queueSize, + boolean hasTimeout, + long timeout, + int queryPriority, + int parallelism, + int yieldAfter, + int batchSize + ) + { + this.workerPool = workerPool; + this.baseSequences = baseSequences; + this.orderingFn = orderingFn; + this.combineFn = combineFn; + this.queueSize = queueSize; + this.hasTimeout = hasTimeout; + this.timeoutAt = System.currentTimeMillis() + timeout; + this.queryPriority = queryPriority; + this.parallelism = parallelism; + this.yieldAfter = yieldAfter; + this.batchSize = batchSize; + } + + @Override + public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + { + final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); + + // 2 layer parallel merge done entirely in fjp + List>> yielders = + baseSequences.stream().map(s -> OrderedResultBatch.fromSequence(s, batchSize)).collect(Collectors.toList()); + + ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( + yielders, + orderingFn, + combineFn, + outputQueue, + queueSize, + parallelism, + yieldAfter, + batchSize, + hasTimeout, + timeoutAt + ); + workerPool.invoke(finalMergeAction); + Sequence finalOutSequence = makeOutputSequenceForQueue(finalMergeAction, outputQueue, hasTimeout, timeoutAt); + return finalOutSequence.toYielder(initValue, accumulator); + } + + /** + * Create an output {@link Sequence} that wraps the output {@link BlockingQueue} of a {@link RecursiveAction} task + */ + private static Sequence makeOutputSequenceForQueue( + RecursiveAction task, + BlockingQueue> queue, + boolean hasTimeout, + long timeoutAt + ) + { + final Sequence backgroundCombineSequence = new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + private OrderedResultBatch currentBatch; + + @Override + public boolean hasNext() + { + if (currentBatch != null && !currentBatch.isTerminalResult() && !currentBatch.isDrained()) { + return true; + } + try { + if (currentBatch == null || currentBatch.isDrained()) { + if (hasTimeout) { + final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); + currentBatch = queue.poll(thisTimeout, TimeUnit.MILLISECONDS); + } else { + currentBatch = queue.take(); + } + } + + if (currentBatch == null) { + throw new RuntimeException(new TimeoutException()); + } + if (currentBatch.isTerminalResult()) { + return false; + } + return true; + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public T next() + { + if (currentBatch == null || currentBatch.isDrained() || currentBatch.isTerminalResult()) { + throw new NoSuchElementException(); + } + return currentBatch.next(); + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + try { + // hmm.. this probably does nothing since this is only the first task and it doesn't run until all + // recursive tasks that are spawned complete + if (!task.isDone()) { + task.cancel(true); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + } + ); + + return backgroundCombineSequence; + } + + /** + * This {@link RecursiveAction} is the initial task of the parallel merge-combine process, it will partition the + * batched result yielders to do 2 layer parallel merge, spawning some number of {@link MergeCombineAction} directly + * for the yielders for the first layer, and spawning a single {@link FinalMergeCombineSeedAction} to wait for results + * to be available in the 'output' {@link BlockingQueue} of the first layer to do a final merge combine of all the + * parallel computed results. + */ + private static class ParallelMergeCombineAction extends RecursiveAction + { + private final List>> yielders; + private final Ordering orderingFn; + private final BinaryOperator combineFn; + private final BlockingQueue> 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 ParallelMergeCombineAction( + List>> yielders, + Ordering orderingFn, + BinaryOperator combineFn, + BlockingQueue> out, + int queueSize, + int parallelism, + int yieldAfter, + int batchSize, + boolean hasTimeout, + long timeoutAt + ) + { + this.yielders = yielders; + 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; + } + + @Override + protected void compute() + { + final int parallelMergeTasks = computeNumTasks(); + + // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, + // just serially perform the merge-combine with a single task + if (yielders.size() < 4 || parallelMergeTasks < 2) { + LOG.debug( + "Input sequence count: %s or available parallel merge task count: %s too small to perform parallel" + + " merge-combine, performing serially with a single merge-combine task", + yielders.size(), + parallelMergeTasks + ); + final PriorityQueue> mergeQueue = new PriorityQueue<>(yielders.size()); + + for (Yielder> s : yielders) { + YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( + s, + orderingFn + ); + mergeQueue.offer(batchedSequenceYielder); + } + + QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + MergeCombineAction mergeAction = new MergeCombineAction( + mergeQueue, + resultsPusher, + orderingFn, + combineFn, + null, + yieldAfter, + batchSize + ); + getPool().execute(mergeAction); + } else { + LOG.debug("Spawning %s parallel merge-combine tasks", parallelMergeTasks); + spawnParallelTasks(parallelMergeTasks); + } + } + + int computeNumTasks() + { + final int parallelTasks = Math.max(2, parallelism - 1); + return Math.min((int) Math.floor((double) yielders.size() / 2.0), parallelTasks); + } + + void spawnParallelTasks(int parallelMergeTasks) + { + List tasks = new ArrayList<>(); + List>> intermediaryOutputs = new ArrayList<>(parallelMergeTasks); + + List>>> partitions = + Lists.partition(yielders, yielders.size() / parallelMergeTasks); + + + for (List>> partition : partitions) { + final PriorityQueue> mergeQueue = new PriorityQueue<>(yielders.size()); + for (Yielder> yielder : partition) { + mergeQueue.offer(new YielderBatchedResultsCursor<>(yielder, orderingFn)); + } + + BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); + intermediaryOutputs.add(outputQueue); + QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); + + MergeCombineAction mergeAction = new MergeCombineAction( + mergeQueue, + pusher, + orderingFn, + combineFn, + null, + yieldAfter, + batchSize + ); + tasks.add(mergeAction); + } + + invokeAll(tasks); + QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + FinalMergeCombineSeedAction finalMergeAction = new FinalMergeCombineSeedAction<>( + intermediaryOutputs, + outputPusher, + orderingFn, + combineFn, + yieldAfter, + batchSize, + hasTimeout, + timeoutAt + ); + + getPool().execute(finalMergeAction); + } + } + + + /** + * This {@link RecursiveAction} is the work-horse of the {@link ParallelMergeCombiningSequence}, it merge-combines + * a set of {@link BatchedResultsCursor} and produces output to a {@link BlockingQueue} with the help of a + * {@link QueuePusher}. This is essentially a composite of logic taken from {@link MergeSequence} and + * {@link org.apache.druid.common.guava.CombiningSequence}, where the {@link Ordering} is used to both set the sort + * order for a {@link PriorityQueue}, and as a comparison to determine if 'same' ordered results need to be combined + * with a supplied {@link BinaryOperator} combining function. + * + * This task takes a 'yieldAfter' parameter which controls how many input result rows will be processed before this + * task completes and executes a new task to continue where it left off. This value is initially set by the + * {@link ParallelMergeCombineAction} to a default value, but after that this process is timed to try and compute + * an 'optimal' number of rows to yield to achieve a task runtime of ~10ms, on the assumption that the time to process + * n results will be approximately the same. + */ + private static class MergeCombineAction extends RecursiveAction + { + private final PriorityQueue> pQueue; + private final Ordering orderingFn; + private final BinaryOperator combineFn; + private final QueuePusher> outputQueue; + private final T initialValue; + private final int yieldAfter; + private final int batchSize; + + private MergeCombineAction( + PriorityQueue> pQueue, + QueuePusher> outputQueue, + Ordering orderingFn, + BinaryOperator combineFn, + T initialValue, + int yieldAfter, + int batchSize + ) + { + this.pQueue = pQueue; + this.orderingFn = orderingFn; + this.combineFn = combineFn; + this.outputQueue = outputQueue; + this.initialValue = initialValue; + this.yieldAfter = yieldAfter; + this.batchSize = batchSize; + } + + @Override + protected void compute() + { + long start = System.nanoTime(); + + int counter = 0; + int batchCounter = 0; + OrderedResultBatch outputBatch = new OrderedResultBatch<>(batchSize); + + T currentCombinedValue = initialValue; + while (counter++ < yieldAfter && !pQueue.isEmpty()) { + + BatchedResultsCursor cursor = pQueue.poll(); + // get the next value to accumulate + + // push the queue along + if (!cursor.isDone()) { + T nextValueToAccumulate = cursor.get(); + + cursor.advance(); + if (!cursor.isDone()) { + pQueue.offer(cursor); + } else { + cursor.close(); + } + + // if current value is null, combine null with next value + if (currentCombinedValue == null) { + currentCombinedValue = combineFn.apply(null, nextValueToAccumulate); + continue; + } + + // if current value is "same" as next value, combine them + if (orderingFn.compare(currentCombinedValue, nextValueToAccumulate) == 0) { + currentCombinedValue = combineFn.apply(currentCombinedValue, nextValueToAccumulate); + continue; + } + + // else, push accumulated value to the queue, accumulate again with next value as initial + if (batchCounter < batchSize) { + outputBatch.add(currentCombinedValue); + batchCounter++; + } else { + outputQueue.offer(outputBatch); + outputBatch = new OrderedResultBatch<>(batchSize); + outputBatch.add(currentCombinedValue); + batchCounter = 1; + } + + // next value is now current value + currentCombinedValue = combineFn.apply(null, nextValueToAccumulate); + } else { + cursor.close(); + } + } + + if (!pQueue.isEmpty()) { + // if there is still work to be done, execute a new task with the current accumulated value to continue + // combining where we left off + if (!outputBatch.isDrained()) { + outputQueue.offer(outputBatch); + } + final long elapsedMillis = Math.max( + TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS), + 1L + ); + final int nextYieldAfter = Math.max(Ints.checkedCast(10 * (yieldAfter / elapsedMillis)), 1); + LOG.debug( + "%s yielded results ran for %s millis, next task yielding every %s operations", + yieldAfter, + elapsedMillis, + nextYieldAfter + ); + getPool().execute(new MergeCombineAction<>( + pQueue, + outputQueue, + orderingFn, + combineFn, + currentCombinedValue, + nextYieldAfter, + batchSize + )); + } else { + // if priority queue is empty, push the final accumulated value + outputBatch.add(currentCombinedValue); + outputQueue.offer(outputBatch); + // ... and the terminal value to indicate the blocking queue holding the values is complete + outputQueue.offer(new OrderedResultBatch<>()); + } + } + } + + /** + * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link BlockingQueue} + * in order to construct a set of {@link BlockingQueueuBatchedResultsCursor} to feed to a {@link MergeCombineAction} + * which will do the actual work of merging and combining the result batches. This is not needed for + * {@link OrderedResultBatch} that are provided by a {@link Yielder} because they will have the initial result batch + * available by virtue of translating the sequence. + */ + private static class FinalMergeCombineSeedAction extends RecursiveAction + { + private final List>> queues; + private final Ordering orderingFn; + private final BinaryOperator combineFn; + private final QueuePusher> outputQueue; + private final int yieldAfter; + private final int batchSize; + private final boolean hasTimeout; + private final long timeoutAt; + + private FinalMergeCombineSeedAction( + List>> queues, + QueuePusher> outputQueue, + Ordering orderingFn, + BinaryOperator combineFn, + int yieldAfter, + int batchSize, + boolean hasTimeout, + long timeoutAt + ) + { + this.queues = queues; + this.orderingFn = orderingFn; + this.combineFn = combineFn; + this.outputQueue = outputQueue; + this.yieldAfter = yieldAfter; + this.batchSize = batchSize; + this.hasTimeout = hasTimeout; + this.timeoutAt = timeoutAt; + } + + @Override + protected void compute() + { + PriorityQueue> cursors = new PriorityQueue<>(queues.size()); + for (BlockingQueue> queue : queues) { + BatchedResultsCursor outputCursor = + new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt); + // seed the Drainer before we can add to pQueue, this is because a blocking queue starts empty where + // a yielder starts with the first value + + outputCursor.initialize(); + cursors.offer(outputCursor); + } + + getPool().execute(new MergeCombineAction( + cursors, + outputQueue, + orderingFn, + combineFn, + null, + yieldAfter, + batchSize + )); + } + } + + /** + * {@link ForkJoinPool} friendly {@link BlockingQueue} feeder, adapted from 'QueueTaker' of Java documentation on + * {@link ForkJoinPool.ManagedBlocker}, + * https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ForkJoinPool.ManagedBlocker.html + */ + static class QueuePusher implements ForkJoinPool.ManagedBlocker + { + final boolean hasTimeout; + final long timeoutAt; + final BlockingQueue queue; + volatile E item = null; + + QueuePusher(BlockingQueue q, boolean hasTimeout, long timeoutAt) + { + this.queue = q; + this.hasTimeout = hasTimeout; + this.timeoutAt = timeoutAt; + } + + @Override + public boolean block() throws InterruptedException + { + if (item != null) { + if (hasTimeout) { + final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); + queue.offer(item, thisTimeout, TimeUnit.MILLISECONDS); + } else { + queue.offer(item); + } + item = null; + } + return true; + } + + @Override + public boolean isReleasable() + { + return item == null; + } + + void addItem(E item) + { + this.item = item; + } + + + public void offer(E item) + { + try { + addItem(item); + ForkJoinPool.managedBlock(this); + } + catch (InterruptedException e) { + throw new RuntimeException("Failed to offer result to output queue", e); + } + } + } + + + /** + * Holder object for an ordered batch of results from a sequence. Batching the results vastly reduces the amount of + * blocking that is needed to move results between stages of {@link MergeCombineAction} done in parallel, allowing + * the fork join tasks to focus on doing actual work instead of dealing with managed blocking. + */ + private static class OrderedResultBatch + { + @Nullable + private final Queue values; + private final boolean isTerminal; + + private OrderedResultBatch(int batchSize) + { + this.values = new ArrayDeque<>(batchSize); + this.isTerminal = false; + } + + private OrderedResultBatch() + { + this.values = null; + this.isTerminal = true; + } + + public void add(E in) + { + assert values != null; + values.offer(in); + } + + public E get() + { + assert values != null; + return values.peek(); + } + + public E next() + { + assert values != null; + return values.poll(); + } + + public boolean isDrained() + { + return !isTerminal && values.isEmpty(); + } + + public boolean isTerminalResult() + { + return isTerminal; + } + + /** + * Convert sequence to yielder that accumulates results into ordered 'batches' + */ + static Yielder> fromSequence(Sequence sequence, int batchSize) + { + return sequence.toYielder( + new OrderedResultBatch<>(batchSize), + new YieldingAccumulator, E>() + { + int count = 0; + @Override + public OrderedResultBatch accumulate(OrderedResultBatch accumulated, E in) + { + count++; + if (/*count == 1 ||*/ count % batchSize == 0) { + yield(); + } + accumulated.add(in); + return accumulated; + } + } + ); + } + } + + /** + * Provides a higher level cursor interface to provide individual results out {@link OrderedResultBatch} provided by + * a {@link Yielder} or {@link BlockingQueue}. This is the mechanism that powers {@link MergeCombineAction}, where + * a set of {@link BatchedResultsCursor} are placed in a {@link PriorityQueue} to facilitate ordering to merge results + * from these cursors, and combine results with the same ordering using the combining function. + */ + abstract static class BatchedResultsCursor + implements ForkJoinPool.ManagedBlocker, Comparable> + { + final Ordering ordering; + volatile OrderedResultBatch resultBatch; + + BatchedResultsCursor(Ordering ordering) + { + this.ordering = ordering; + } + + public void initialize() + { + // nothing to initialize for yielders since they come primed, blocking queue will need to block for some data + // though so it is ready to go + } + + public abstract void advance(); + + public void nextBatch() + { + try { + ForkJoinPool.managedBlock(this); + } + catch (InterruptedException e) { + throw new RuntimeException("Failed to load next batch of results", e); + } + } + + public abstract boolean isDone(); + + public void close() + { + // nothing to close for blocking queue, but yielders will need to clean up or they will leak resources + } + + public E get() + { + return resultBatch.get(); + } + + @Override + public int compareTo(BatchedResultsCursor o) + { + return ordering.compare(get(), o.get()); + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof ParallelMergeCombiningSequence.BatchedResultsCursor)) { + return false; + } + return compareTo((BatchedResultsCursor) o) == 0; + } + + @Override + public int hashCode() + { + return Objects.hash(ordering); + } + } + + /** + * {@link BatchedResultsCursor} that wraps a {@link Yielder} of {@link OrderedResultBatch} to provide individual rows + * of the result batch. + */ + static class YielderBatchedResultsCursor extends BatchedResultsCursor + { + Yielder> yielder; + + YielderBatchedResultsCursor(Yielder> yielder, Ordering ordering) + { + super(ordering); + this.yielder = yielder; + resultBatch = yielder.get(); + } + + @Override + public void advance() + { + if (!resultBatch.isDrained()) { + resultBatch.next(); + } + if (resultBatch.isDrained() && !yielder.isDone()) { + nextBatch(); + } + } + + @Override + public boolean isDone() + { + // yielder will never produce a 'terminal' result batch, so only check that we drain the final batch when the + // yielder is done + return resultBatch == null || (yielder.isDone() && resultBatch.isDrained()); + } + + @Override + public boolean block() + { + if (yielder.isDone()) { + return true; + } + if (resultBatch == null || resultBatch.isDrained()) { + final Yielder> nextYielder = yielder.next(resultBatch); + yielder = nextYielder; + } + return true; + } + + @Override + public boolean isReleasable() + { + return resultBatch != null && !resultBatch.isDrained(); + } + + @Override + public void close() + { + try { + yielder.close(); + } + catch (IOException e) { + throw new RuntimeException("Failed to close yielder", e); + } + } + } + + /** + * {@link BatchedResultsCursor} that wraps a {@link BlockingQueue} of {@link OrderedResultBatch} to provide individual + * rows from the result batch. + */ + static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor + { + final BlockingQueue> queue; + final boolean hasTimeout; + final long timeoutAt; + + BlockingQueueuBatchedResultsCursor( + BlockingQueue> blockingQueue, + Ordering ordering, + boolean hasTimeout, + long timeoutAt + ) + { + super(ordering); + this.queue = blockingQueue; + this.hasTimeout = hasTimeout; + this.timeoutAt = timeoutAt; + } + + @Override + public void initialize() + { + nextBatch(); + } + + @Override + public void advance() + { + if (!resultBatch.isDrained()) { + resultBatch.next(); + } + if (resultBatch.isDrained()) { + nextBatch(); + } + } + + @Override + public boolean isDone() + { + // blocking queue cursors always will finish the queue with a 'terminal' result batch to indicate that the queue + // is finished and no additional values are expected. + return resultBatch.isTerminalResult(); + } + + @Override + public boolean block() throws InterruptedException + { + if (resultBatch == null || resultBatch.isDrained()) { + if (hasTimeout) { + final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); + resultBatch = queue.poll(thisTimeout, TimeUnit.MILLISECONDS); + } else { + resultBatch = queue.take(); + } + } + return true; + } + + @Override + public boolean isReleasable() + { + // if result batch is 'terminal' or still has values, no need to block + if (resultBatch != null && (resultBatch.isTerminalResult() || !resultBatch.isDrained())) { + return true; + } + // if we can get a result immediately without blocking, also no need to block + resultBatch = queue.poll(); + return resultBatch != null; + } + } +} diff --git a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java index ba5504f25cf1..c6cf2cb3266a 100644 --- a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java +++ b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java @@ -36,7 +36,7 @@ public class ComplexSequenceTest { // Integer::sum with more nulls - private static final BinaryOperator PLUS_NULLABLE = (arg1, arg2) -> { + public static final BinaryOperator PLUS_NULLABLE = (arg1, arg2) -> { if (arg1 == null) { return arg2; } diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java new file mode 100644 index 000000000000..047b9c894296 --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -0,0 +1,236 @@ +/* + * 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.Ordering; +import org.apache.druid.common.guava.CombiningSequence; +import org.apache.druid.java.util.common.Pair; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ForkJoinPool; +import java.util.function.BinaryOperator; + +public class ParallelMergeCombiningSequenceTest +{ + private ForkJoinPool pool; + + @Before + public void setup() + { + pool = new ForkJoinPool(4, ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true); + } + + @After + public void teardown() + { + pool.shutdown(); + } + + @Test + public void testSimple() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 1) + ); + + List pairs2 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 2) + ); + + + List> input = new ArrayList<>(); + input.add(Sequences.simple(pairs1)); + input.add(Sequences.simple(pairs2)); + + assertResult(input); + } + + @Test + public void testLongBoy() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 11), + new IntPair(7, 11), + new IntPair(9, 11), + new IntPair(11, 11), + new IntPair(16, 11), + new IntPair(24, 11), + new IntPair(25, 11), + new IntPair(27, 11), + new IntPair(28, 1) + ); + + List pairs2 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 1), + new IntPair(7, 1), + new IntPair(9, 1), + new IntPair(10, 1), + new IntPair(13, 1), + new IntPair(14, 1), + new IntPair(23, 1), + new IntPair(25, 1), + new IntPair(27, 1), + new IntPair(28, 2) + ); + + + List> input = new ArrayList<>(); + input.add(Sequences.simple(pairs1)); + input.add(Sequences.simple(pairs2)); + + assertResult(input); + } + + @Test + public void testSomeStuff() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 1) + ); + + List pairs2 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(6, 2), + new IntPair(10, 2) + ); + + List pairs3 = Arrays.asList( + new IntPair(4, 5), + new IntPair(10, 3) + ); + + List pairs4 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(6, 2), + new IntPair(10, 2) + ); + + List pairs5 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 1) + ); + + List> input = new ArrayList<>(); + input.add(Sequences.simple(pairs1)); + input.add(Sequences.simple(pairs2)); + input.add(Sequences.simple(pairs3)); + input.add(Sequences.simple(pairs4)); + input.add(Sequences.simple(pairs5)); + + assertResult(input); + } + + private void assertResult(List> sequences) throws InterruptedException + { + final Ordering ordering = Ordering.natural().onResultOf(p -> p.lhs); + final BinaryOperator mergeFn = (lhs, rhs) -> { + if (lhs == null) { + return rhs; + } + + if (rhs == null) { + return lhs; + } + + return new IntPair(lhs.lhs, lhs.rhs + rhs.rhs); + }; + + final CombiningSequence combiningSequence = CombiningSequence.create( + new MergeSequence<>(ordering, Sequences.simple(sequences)), + ordering, + mergeFn + ); + + final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( + pool, + sequences, + ordering, + mergeFn, + 64, + false, + 5000, + 0, + Runtime.getRuntime().availableProcessors() - 1, + 8, + 4 + ); + + Yielder combiningYielder = Yielders.each(combiningSequence); + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + + IntPair prev = null; + + while (!combiningYielder.isDone() && !parallelMergeCombineYielder.isDone()) { + Assert.assertEquals(combiningYielder.get(), parallelMergeCombineYielder.get()); + System.out.println(parallelMergeCombineYielder.get()); + Assert.assertNotEquals(parallelMergeCombineYielder.get(), prev); + prev = parallelMergeCombineYielder.get(); + combiningYielder = combiningYielder.next(combiningYielder.get()); + parallelMergeCombineYielder = parallelMergeCombineYielder.next(parallelMergeCombineYielder.get()); + } + + Assert.assertTrue(combiningYielder.isDone()); + Assert.assertTrue(parallelMergeCombineYielder.isDone()); + while (pool.getRunningThreadCount() > 0) { + Thread.sleep(100); + } + Assert.assertEquals(0, pool.getRunningThreadCount()); + } + + static class IntPair extends Pair + { + IntPair(@Nullable Integer lhs, @Nullable Integer rhs) + { + super(lhs, rhs); + } + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 762f975b36a1..fadd572eb361 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -41,6 +41,7 @@ import org.apache.druid.data.input.Row; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.http.DruidHttpClientConfig; @@ -84,6 +85,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; /** * Base class for implementing MovingAverageQuery tests @@ -349,7 +351,8 @@ public long getMaxQueuedBytes() { return 0L; } - } + }, + new LifecycleForkJoinPool(6, ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true, 1000L) ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java new file mode 100644 index 000000000000..cf6dcca9ed53 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java @@ -0,0 +1,61 @@ +/* + * 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.guice; + + +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; + +public class LifecycleForkJoinPool extends ForkJoinPool +{ + private static final Logger LOG = new Logger(LifecycleForkJoinPool.class); + private final long awaitShutdownMillis; + + public LifecycleForkJoinPool( + int parallelism, + ForkJoinWorkerThreadFactory factory, + Thread.UncaughtExceptionHandler handler, + boolean asyncMode, + long awaitShutdownMillis + ) + { + super(parallelism, factory, handler, asyncMode); + this.awaitShutdownMillis = awaitShutdownMillis; + } + + @LifecycleStop + public void stop() + { + LOG.info("Shutting down ForkJoinPool [%s]", this); + shutdown(); + try { + if (!awaitTermination(awaitShutdownMillis, TimeUnit.MILLISECONDS)) { + LOG.warn("Failed to complete all tasks in FJP [%s]", this); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("interrupted on shutdown", e); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index c9afa1ab3c67..cc48de883e07 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -34,6 +34,7 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem public static final int DEFAULT_NUM_MERGE_BUFFERS = -1; public static final int DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = -1; public static final int MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024; + public static final int DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS = 60_000; private AtomicReference computedBufferSizeBytes = new AtomicReference<>(); @@ -144,4 +145,26 @@ public String getTmpDir() { return System.getProperty("java.io.tmpdir"); } + + @Config(value = "${base_path}.numMergePoolThreads") + public int getNumThreadsMergePoolConfigured() + { + return DEFAULT_NUM_THREADS; + } + + public int getNumThreadsMergePool() + { + int numThreadsConfigured = getNumThreadsMergePoolConfigured(); + if (numThreadsConfigured != DEFAULT_NUM_THREADS) { + return numThreadsConfigured; + } else { + return (int) Math.ceil(JvmUtils.getRuntimeInfo().getAvailableProcessors() * 1.5); + } + } + + @Config(value = "${base_path}.mergePoolAwaitShutdownMillis") + public long getMergePoolAwaitShutdownMillis() + { + return DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS; + } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index d5003e26cca3..0a2d362ee016 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -38,6 +38,11 @@ public class QueryContexts public static final String MAX_SCATTER_GATHER_BYTES_KEY = "maxScatterGatherBytes"; public static final String MAX_QUEUED_BYTES_KEY = "maxQueuedBytes"; public static final String DEFAULT_TIMEOUT_KEY = "defaultTimeout"; + public static final String BROKER_PARALLEL_MERGE_KEY = "enableParallelMerge"; + public static final String BROKER_PARALLEL_MERGE_INITIAL_YIELD_ROWS_KEY = "parallelMergeInitialYieldRows"; + public static final String BROKER_PARALLEL_MERGE_SMALL_BATCH_ROWS_KEY = "parallelMergeSmallBatchRows"; + public static final String BROKER_PARALLELISM = "parallelMergeParallelism"; + @Deprecated public static final String CHUNK_PERIOD_KEY = "chunkPeriod"; @@ -51,6 +56,10 @@ public class QueryContexts public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0; public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); public static final long NO_TIMEOUT = 0; + public static final boolean DEFAULT_ENABLE_PARALLEL_MERGE = false; + public static final int DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS = 1024; + public static final int DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS = 128; + public static final int DEFAULT_PARALLEL_MERGE_PARALLELISM = 3; @SuppressWarnings("unused") // Used by Jackson serialization public enum Vectorize @@ -179,6 +188,26 @@ public static int getPriority(Query query, int defaultValue) return parseInt(query, PRIORITY_KEY, defaultValue); } + public static boolean getEnableParallelMerges(Query query) + { + return parseBoolean(query, BROKER_PARALLEL_MERGE_KEY, DEFAULT_ENABLE_PARALLEL_MERGE); + } + + public static int getParallelMergeInitialYieldRows(Query query) + { + return parseInt(query, BROKER_PARALLEL_MERGE_INITIAL_YIELD_ROWS_KEY, DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS); + } + + public static int getParallelMergeSmallBatchRows(Query query) + { + return parseInt(query, BROKER_PARALLEL_MERGE_SMALL_BATCH_ROWS_KEY, DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS); + } + + public static int getParallelMergeParallelism(Query query) + { + return parseInt(query, BROKER_PARALLELISM, DEFAULT_PARALLEL_MERGE_PARALLELISM); + } + @Deprecated public static String getChunkPeriod(Query query) { diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 18a4a028b367..971276d7fee5 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -36,6 +36,7 @@ import org.apache.druid.client.cache.CachePopulator; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.annotations.Client; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.guice.http.DruidHttpClientConfig; @@ -45,6 +46,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.LazySequence; +import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -102,6 +104,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final CachePopulator cachePopulator; private final CacheConfig cacheConfig; private final DruidHttpClientConfig httpClientConfig; + private final LifecycleForkJoinPool pool; @Inject public CachingClusteredClient( @@ -111,7 +114,8 @@ public CachingClusteredClient( @Smile ObjectMapper objectMapper, CachePopulator cachePopulator, CacheConfig cacheConfig, - @Client DruidHttpClientConfig httpClientConfig + @Client DruidHttpClientConfig httpClientConfig, + LifecycleForkJoinPool pool ) { this.warehouse = warehouse; @@ -121,6 +125,7 @@ public CachingClusteredClient( this.cachePopulator = cachePopulator; this.cacheConfig = cacheConfig; this.httpClientConfig = httpClientConfig; + this.pool = pool; if (cacheConfig.isQueryCacheable(Query.GROUP_BY) && (cacheConfig.isUseCache() || cacheConfig.isPopulateCache())) { log.warn( @@ -286,10 +291,33 @@ Sequence run(final UnaryOperator> time List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); addSequencesFromServer(sequencesByInterval, segmentsByServer); + return merge(sequencesByInterval); + }); + } + + private Sequence merge(List> sequencesByInterval) + { + if (QueryContexts.getEnableParallelMerges(query)) { + final int yieldAfter = QueryContexts.getParallelMergeInitialYieldRows(query); + final int batchSize = QueryContexts.getParallelMergeSmallBatchRows(query); + return new ParallelMergeCombiningSequence<>( + pool, + sequencesByInterval, + query.getResultOrdering(), + toolChest.createMergeFn(query), + 8 * (yieldAfter / batchSize), + QueryContexts.hasTimeout(query), + QueryContexts.getTimeout(query), + QueryContexts.getPriority(query), + QueryContexts.getParallelMergeParallelism(query), + yieldAfter, + batchSize + ); + } else { return Sequences .simple(sequencesByInterval) .flatMerge(seq -> seq, query.getResultOrdering()); - }); + } } private Set computeSegmentsToQuery(TimelineLookup timeline) diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index efb140112d26..a9eee92b34d2 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -53,6 +53,7 @@ import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ForkJoinPool; /** */ @@ -135,6 +136,19 @@ public BlockingPool getMergeBufferPool(DruidProcessingConfig config) ); } + @Provides + @ManageLifecycle + public LifecycleForkJoinPool getMergeProcessingPool(DruidProcessingConfig config) + { + return new LifecycleForkJoinPool( + config.getNumThreadsMergePool(), + ForkJoinPool.defaultForkJoinWorkerThreadFactory, // todo: ? + null, + true, + config.getMergePoolAwaitShutdownMillis() + ); + } + private void verifyDirectMemory(DruidProcessingConfig config) { try { diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 9a7bb900a5fa..be7ede0c2cfa 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -35,6 +35,7 @@ import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.client.selector.TierSelectorStrategy; +import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -68,6 +69,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; /** */ @@ -313,7 +315,8 @@ public long getMaxQueuedBytes() { return 0L; } - } + }, + new LifecycleForkJoinPool(6, ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true, 1000L) ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 2b8e2f52b811..72eae6719ca7 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -52,6 +52,7 @@ import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.RandomServerSelectorStrategy; import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; @@ -163,6 +164,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; /** * @@ -2756,7 +2758,14 @@ public long getMaxQueuedBytes() { return 0L; } - } + }, + new LifecycleForkJoinPool( + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + ForkJoinPool.defaultForkJoinWorkerThreadFactory, + null, + true, + 1000L + ) ); } From 7320b93dd1fc6b65d459952bfde35fb3f227ca87 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Sep 2019 19:18:41 -0700 Subject: [PATCH 02/51] fix non-terminating sequences, auto compute parallelism --- .../guava/ParallelMergeCombiningSequence.java | 88 +++++--- .../ParallelMergeCombiningSequenceTest.java | 189 +++++++++++++++++- 2 files changed, 248 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 83cb23dc6191..ef133cfe9a40 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -23,6 +23,7 @@ import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.io.IOException; @@ -94,25 +95,31 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat { final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); - // 2 layer parallel merge done entirely in fjp List>> yielders = - baseSequences.stream().map(s -> OrderedResultBatch.fromSequence(s, batchSize)).collect(Collectors.toList()); - - ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( - yielders, - orderingFn, - combineFn, - outputQueue, - queueSize, - parallelism, - yieldAfter, - batchSize, - hasTimeout, - timeoutAt - ); - workerPool.invoke(finalMergeAction); - Sequence finalOutSequence = makeOutputSequenceForQueue(finalMergeAction, outputQueue, hasTimeout, timeoutAt); - return finalOutSequence.toYielder(initValue, accumulator); + baseSequences.stream() + .map(s -> OrderedResultBatch.fromSequence(s, batchSize)) + .filter(y -> !(y.get() == null || (y.get().isDrained() && y.isDone()))) + .collect(Collectors.toList()); + if (yielders.size() > 0) { + // 2 layer parallel merge done in fjp + ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( + yielders, + orderingFn, + combineFn, + outputQueue, + queueSize, + parallelism, + yieldAfter, + batchSize, + hasTimeout, + timeoutAt + ); + workerPool.execute(finalMergeAction); + Sequence finalOutSequence = makeOutputSequenceForQueue(finalMergeAction, outputQueue, hasTimeout, timeoutAt); + return finalOutSequence.toYielder(initValue, accumulator); + } + // empty result + return Sequences.empty().toYielder(initValue, accumulator); } /** @@ -179,8 +186,9 @@ public T next() public void cleanup(Iterator iterFromMake) { try { - // hmm.. this probably does nothing since this is only the first task and it doesn't run until all - // recursive tasks that are spawned complete + // todo: hmm.. this probably does nothing since this is only the first task and it doesn't run until all + // recursive tasks that are spawned complete, maybe we need a collection of 'active' tasks which the tasks + // themselves update? if (!task.isDone()) { task.cancel(true); } @@ -248,7 +256,7 @@ protected void compute() // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, // just serially perform the merge-combine with a single task if (yielders.size() < 4 || parallelMergeTasks < 2) { - LOG.debug( + LOG.info( "Input sequence count: %s or available parallel merge task count: %s too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", yielders.size(), @@ -276,15 +284,42 @@ protected void compute() ); getPool().execute(mergeAction); } else { - LOG.debug("Spawning %s parallel merge-combine tasks", parallelMergeTasks); + LOG.info("Spawning %s parallel merge-combine tasks", parallelMergeTasks); spawnParallelTasks(parallelMergeTasks); } } int computeNumTasks() { - final int parallelTasks = Math.max(2, parallelism - 1); - return Math.min((int) Math.floor((double) yielders.size() / 2.0), parallelTasks); + final int numProcessors = JvmUtils.getRuntimeInfo().getAvailableProcessors(); + final int poolParallelism = getPool().getParallelism(); + final int activeThreadCount = getPool().getActiveThreadCount(); + final int runningThreadCount = getPool().getRunningThreadCount(); + final int submissionCount = getPool().getQueuedSubmissionCount(); + final long queuedTaskCount = getPool().getQueuedTaskCount(); + LOG.info( + "processors: [%s] pool parallelism: [%s] active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s]", + numProcessors, + poolParallelism, + activeThreadCount, + runningThreadCount, + submissionCount, + queuedTaskCount + ); + // max is minimum of either number of processors - 1 or user suggested parallelism + final int maxParallelism = Math.min(numProcessors, parallelism); + // adjust max to be no more than total pool parallelism less the number of active threads + final int computedParallelism = Math.min(maxParallelism, poolParallelism - activeThreadCount); + // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot, we need at least 2 + final int computedOptimalParallelism = Math.min((int) Math.floor((double) yielders.size() / 2.0), computedParallelism - 1); + return computedOptimalParallelism; + + + /* original manual control + final int suggestedParallelism = Math.max(2, parallelism - 1); + final int suggestedOptimalParallelism = Math.min((int) Math.floor((double) yielders.size() / 2.0), suggestedParallelism); + return suggestedOptimalParallelism; + */ } void spawnParallelTasks(int parallelMergeTasks) @@ -318,7 +353,10 @@ void spawnParallelTasks(int parallelMergeTasks) tasks.add(mergeAction); } - invokeAll(tasks); + for (RecursiveAction task : tasks) { + getPool().execute(task); + } + QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); FinalMergeCombineSeedAction finalMergeAction = new FinalMergeCombineSeedAction<>( intermediaryOutputs, diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index 047b9c894296..e04aee43b049 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.Ordering; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.logger.Logger; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -36,6 +37,8 @@ public class ParallelMergeCombiningSequenceTest { + private static final Logger LOG = new Logger(ParallelMergeCombiningSequenceTest.class); + private ForkJoinPool pool; @Before @@ -51,7 +54,76 @@ public void teardown() } @Test - public void testSimple() throws Exception + public void testNone() throws Exception + { + List> input = new ArrayList<>(); + assertResult(input); + } + + @Test + public void testEmptySerial() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 1) + ); + List> input = new ArrayList<>(); + input.add(Sequences.empty()); + input.add(Sequences.simple(pairs1)); + assertResult(input); + } + + @Test + public void testEmpty() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 1) + ); + List> input = new ArrayList<>(); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.simple(pairs1)); + assertResult(input); + } + + @Test + public void testSimpleSerial() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 1) + ); + + List pairs2 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 2) + ); + + + List> input = new ArrayList<>(); + input.add(Sequences.simple(pairs1)); + input.add(Sequences.simple(pairs2)); + + assertResult(input); + } + + @Test + public void testSimpleParallel() throws Exception { List pairs1 = Arrays.asList( new IntPair(0, 6), @@ -68,16 +140,32 @@ public void testSimple() throws Exception new IntPair(5, 2) ); + List pairs3 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 2) + ); + + List pairs4 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 2) + ); + List> input = new ArrayList<>(); input.add(Sequences.simple(pairs1)); input.add(Sequences.simple(pairs2)); + input.add(Sequences.simple(pairs3)); + input.add(Sequences.simple(pairs4)); assertResult(input); } @Test - public void testLongBoy() throws Exception + public void testLongBoySerial() throws Exception { List pairs1 = Arrays.asList( new IntPair(0, 6), @@ -111,16 +199,109 @@ public void testLongBoy() throws Exception new IntPair(28, 2) ); + List> input = new ArrayList<>(); + input.add(Sequences.simple(pairs1)); + input.add(Sequences.simple(pairs2)); + + assertResult(input); + } + + @Test + public void testLongBoyParallel() throws Exception + { + List pairs1 = Arrays.asList( + new IntPair(0, 6), + new IntPair(1, 1), + new IntPair(2, 1), + new IntPair(5, 11), + new IntPair(6, 11), + new IntPair(7, 11), + new IntPair(9, 11), + new IntPair(11, 11), + new IntPair(16, 11), + new IntPair(24, 11), + new IntPair(25, 11), + new IntPair(27, 11), + new IntPair(28, 1) + ); + + List pairs2 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 1), + new IntPair(7, 1), + new IntPair(9, 1), + new IntPair(10, 1), + new IntPair(13, 1), + new IntPair(14, 1), + new IntPair(23, 1), + new IntPair(25, 1), + new IntPair(27, 1), + new IntPair(28, 2) + ); + + List pairs3 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 1), + new IntPair(7, 6), + new IntPair(9, 1), + new IntPair(10, 2), + new IntPair(13, 1), + new IntPair(14, 3), + new IntPair(15, 1), + new IntPair(19, 7), + new IntPair(21, 1), + new IntPair(22, 2) + ); + + List pairs4 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 1), + new IntPair(7, 4), + new IntPair(9, 1), + new IntPair(10, 2), + new IntPair(13, 1), + new IntPair(14, 11), + new IntPair(16, 1), + new IntPair(17, 13), + new IntPair(20, 1), + new IntPair(22, 2) + ); + + List pairs5 = Arrays.asList( + new IntPair(0, 1), + new IntPair(1, 13), + new IntPair(4, 1), + new IntPair(5, 1), + new IntPair(7, 1), + new IntPair(9, 1), + new IntPair(10, 1), + new IntPair(13, 1), + new IntPair(14, 1), + new IntPair(16, 1), + new IntPair(25, 7), + new IntPair(27, 1), + new IntPair(30, 2) + ); + List> input = new ArrayList<>(); input.add(Sequences.simple(pairs1)); input.add(Sequences.simple(pairs2)); + input.add(Sequences.simple(pairs3)); + input.add(Sequences.simple(pairs4)); + input.add(Sequences.simple(pairs5)); assertResult(input); } @Test - public void testSomeStuff() throws Exception + public void testMixedLength() throws Exception { List pairs1 = Arrays.asList( new IntPair(0, 6), @@ -211,7 +392,7 @@ private void assertResult(List> sequences) throws InterruptedE while (!combiningYielder.isDone() && !parallelMergeCombineYielder.isDone()) { Assert.assertEquals(combiningYielder.get(), parallelMergeCombineYielder.get()); - System.out.println(parallelMergeCombineYielder.get()); + LOG.info("%s", parallelMergeCombineYielder.get()); Assert.assertNotEquals(parallelMergeCombineYielder.get(), prev); prev = parallelMergeCombineYielder.get(); combiningYielder = combiningYielder.next(combiningYielder.get()); From fb62358c2677c9d36d5ea5519986657f542b6e12 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 12 Sep 2019 01:27:46 -0700 Subject: [PATCH 03/51] adjust benches --- .../benchmark/query/CachingClusteredClientBenchmark.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index ba6ead5e51b8..c2871482f92e 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -148,11 +148,11 @@ public class CachingClusteredClientBenchmark private static final String DATA_SOURCE = "ds"; public static final ObjectMapper JSON_MAPPER; - @Param({"8", "24"}) + @Param({"8", "24", "64"}) private int numServers; - @Param({"5"}) + @Param({"1", "7"}) private int parallelism; @Param({"parallel", "serial"}) From 1054d31db66495848fa2b5eb6b6c60ed56a126fc Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 16 Sep 2019 13:55:42 -0700 Subject: [PATCH 04/51] adjust benchmarks --- .../query/CachingClusteredClientBenchmark.java | 18 +++--------------- .../guava/ParallelMergeCombiningSequence.java | 2 +- 2 files changed, 4 insertions(+), 16 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index c2871482f92e..afdb76a2d5ee 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -148,16 +148,13 @@ public class CachingClusteredClientBenchmark private static final String DATA_SOURCE = "ds"; public static final ObjectMapper JSON_MAPPER; - @Param({"8", "24", "64"}) + @Param({"8", "32"}) private int numServers; - @Param({"1", "7"}) + @Param({"0", "1", "7"}) private int parallelism; - @Param({"parallel", "serial"}) - private String mergeCombineStyle; - @Param({"75000"}) private int rowsPerSegment; @@ -171,7 +168,6 @@ public class CachingClusteredClientBenchmark private LifecycleForkJoinPool forkJoinPool; private boolean parallelCombine; - private boolean possiblyParallel; private Query query; @@ -199,15 +195,7 @@ public void setup() { final String schemaName = "basic"; - switch (mergeCombineStyle) { - case "parallel": - parallelCombine = true; - break; - case "serial": - default: - parallelCombine = false; - break; - } + parallelCombine = parallelism > 0; BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index ef133cfe9a40..80acf93562c0 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -306,7 +306,7 @@ int computeNumTasks() submissionCount, queuedTaskCount ); - // max is minimum of either number of processors - 1 or user suggested parallelism + // max is minimum of either number of processors or user suggested parallelism final int maxParallelism = Math.min(numProcessors, parallelism); // adjust max to be no more than total pool parallelism less the number of active threads final int computedParallelism = Math.min(maxParallelism, poolParallelism - activeThreadCount); From 2764b9c8866bf209db87b8292b7440dedc9bcec7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 20 Sep 2019 20:16:48 -0700 Subject: [PATCH 05/51] now hella more faster, fixed dumb --- .../CachingClusteredClientBenchmark.java | 11 +- .../guava/ParallelMergeCombiningSequence.java | 147 ++++++++++++------ 2 files changed, 105 insertions(+), 53 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index afdb76a2d5ee..f5ba355c0aeb 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -148,17 +148,20 @@ public class CachingClusteredClientBenchmark private static final String DATA_SOURCE = "ds"; public static final ObjectMapper JSON_MAPPER; - @Param({"8", "32"}) + @Param({"8", "32", "64"}) +// @Param({"8"}) private int numServers; - @Param({"0", "1", "7"}) +// @Param({"0", "1", "3", "4", "5", "6", "7", "8"}) + @Param({"0", "1", "4"}) private int parallelism; @Param({"75000"}) private int rowsPerSegment; @Param({"all", "minute"}) +// @Param({"all"}) private String queryGranularity; private QueryToolChestWarehouse toolChestWarehouse; @@ -405,7 +408,7 @@ public void topNQuery(Blackhole blackhole) query = new TopNQueryBuilder() .dataSource(DATA_SOURCE) .intervals(basicSchemaIntervalSpec) - .dimension(new DefaultDimensionSpec("dimSequential", null)) + .dimension(new DefaultDimensionSpec("dimZipf", null)) .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) .granularity(Granularity.fromString(queryGranularity)) .metric("sumLongSequential") @@ -435,7 +438,7 @@ public void groupByQuery(Blackhole blackhole) .setDataSource(DATA_SOURCE) .setQuerySegmentSpec(basicSchemaIntervalSpec) .setDimensions( - new DefaultDimensionSpec("dimSequentialHalfNull", null), + new DefaultDimensionSpec("dimZipf", null), new DefaultDimensionSpec("dimSequential", null) ) .setAggregatorSpecs(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 80acf93562c0..c0276df15d58 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -42,7 +42,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.BinaryOperator; -import java.util.stream.Collectors; /** * Artisanal, locally-sourced, hand-crafted, gluten and GMO free, bespoke, small-batch parallel merge combinining sequence @@ -95,15 +94,10 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat { final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); - List>> yielders = - baseSequences.stream() - .map(s -> OrderedResultBatch.fromSequence(s, batchSize)) - .filter(y -> !(y.get() == null || (y.get().isDrained() && y.isDone()))) - .collect(Collectors.toList()); - if (yielders.size() > 0) { + if (baseSequences.size() > 0) { // 2 layer parallel merge done in fjp ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( - yielders, + baseSequences, orderingFn, combineFn, outputQueue, @@ -206,13 +200,13 @@ public void cleanup(Iterator iterFromMake) /** * This {@link RecursiveAction} is the initial task of the parallel merge-combine process, it will partition the * batched result yielders to do 2 layer parallel merge, spawning some number of {@link MergeCombineAction} directly - * for the yielders for the first layer, and spawning a single {@link FinalMergeCombineSeedAction} to wait for results + * for the yielders for the first layer, and spawning a single {@link BlockingQueueMergeCombineSeedAction} to wait for results * to be available in the 'output' {@link BlockingQueue} of the first layer to do a final merge combine of all the * parallel computed results. */ private static class ParallelMergeCombineAction extends RecursiveAction { - private final List>> yielders; + private final List> sequences; private final Ordering orderingFn; private final BinaryOperator combineFn; private final BlockingQueue> out; @@ -224,7 +218,7 @@ private static class ParallelMergeCombineAction extends RecursiveAction private final long timeoutAt; private ParallelMergeCombineAction( - List>> yielders, + List> sequences, Ordering orderingFn, BinaryOperator combineFn, BlockingQueue> out, @@ -236,7 +230,7 @@ private ParallelMergeCombineAction( long timeoutAt ) { - this.yielders = yielders; + this.sequences = sequences; this.combineFn = combineFn; this.orderingFn = orderingFn; this.out = out; @@ -255,36 +249,26 @@ protected void compute() // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, // just serially perform the merge-combine with a single task - if (yielders.size() < 4 || parallelMergeTasks < 2) { - LOG.info( + if (sequences.size() < 4 || parallelMergeTasks < 2) { + LOG.debug( "Input sequence count: %s or available parallel merge task count: %s too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", - yielders.size(), + sequences.size(), parallelMergeTasks ); - final PriorityQueue> mergeQueue = new PriorityQueue<>(yielders.size()); - - for (Yielder> s : yielders) { - YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( - s, - orderingFn - ); - mergeQueue.offer(batchedSequenceYielder); - } QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); - MergeCombineAction mergeAction = new MergeCombineAction( - mergeQueue, + YielderMergeCombineSeedAction mergeAction = new YielderMergeCombineSeedAction<>( + sequences, resultsPusher, orderingFn, combineFn, - null, yieldAfter, batchSize ); - getPool().execute(mergeAction); + invokeAll(mergeAction); } else { - LOG.info("Spawning %s parallel merge-combine tasks", parallelMergeTasks); + LOG.debug("Spawning %s parallel merge-combine tasks", parallelMergeTasks); spawnParallelTasks(parallelMergeTasks); } } @@ -297,7 +281,7 @@ int computeNumTasks() final int runningThreadCount = getPool().getRunningThreadCount(); final int submissionCount = getPool().getQueuedSubmissionCount(); final long queuedTaskCount = getPool().getQueuedTaskCount(); - LOG.info( + LOG.debug( "processors: [%s] pool parallelism: [%s] active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s]", numProcessors, poolParallelism, @@ -311,7 +295,7 @@ int computeNumTasks() // adjust max to be no more than total pool parallelism less the number of active threads final int computedParallelism = Math.min(maxParallelism, poolParallelism - activeThreadCount); // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot, we need at least 2 - final int computedOptimalParallelism = Math.min((int) Math.floor((double) yielders.size() / 2.0), computedParallelism - 1); + final int computedOptimalParallelism = Math.min((int) Math.floor((double) sequences.size() / 2.0), computedParallelism - 1); return computedOptimalParallelism; @@ -327,38 +311,34 @@ void spawnParallelTasks(int parallelMergeTasks) List tasks = new ArrayList<>(); List>> intermediaryOutputs = new ArrayList<>(parallelMergeTasks); - List>>> partitions = - Lists.partition(yielders, yielders.size() / parallelMergeTasks); + List>> partitions = + Lists.partition(sequences, sequences.size() / parallelMergeTasks); - for (List>> partition : partitions) { - final PriorityQueue> mergeQueue = new PriorityQueue<>(yielders.size()); - for (Yielder> yielder : partition) { - mergeQueue.offer(new YielderBatchedResultsCursor<>(yielder, orderingFn)); - } + for (List> partition : partitions) { BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); intermediaryOutputs.add(outputQueue); QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); - MergeCombineAction mergeAction = new MergeCombineAction( - mergeQueue, + YielderMergeCombineSeedAction mergeAction = new YielderMergeCombineSeedAction( + partition, pusher, orderingFn, combineFn, - null, yieldAfter, batchSize ); tasks.add(mergeAction); } - for (RecursiveAction task : tasks) { - getPool().execute(task); - } +// for (RecursiveAction task : tasks) { +// getPool().execute(task); +// } + invokeAll(tasks); QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); - FinalMergeCombineSeedAction finalMergeAction = new FinalMergeCombineSeedAction<>( + BlockingQueueMergeCombineSeedAction finalMergeAction = new BlockingQueueMergeCombineSeedAction<>( intermediaryOutputs, outputPusher, orderingFn, @@ -509,6 +489,68 @@ protected void compute() } } + /** + * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link BlockingQueue} + * in order to construct a set of {@link YielderBatchedResultsCursor} to feed to a {@link MergeCombineAction} + * which will do the actual work of merging and combining the result batches. + */ + private static class YielderMergeCombineSeedAction extends RecursiveAction + { + private final List> sequences; + private final Ordering orderingFn; + private final BinaryOperator combineFn; + private final QueuePusher> outputQueue; + private final int yieldAfter; + private final int batchSize; + + private YielderMergeCombineSeedAction( + List> sequences, + QueuePusher> outputQueue, + Ordering orderingFn, + BinaryOperator combineFn, + int yieldAfter, + int batchSize + ) + { + this.sequences = sequences; + this.orderingFn = orderingFn; + this.combineFn = combineFn; + this.outputQueue = outputQueue; + this.yieldAfter = yieldAfter; + this.batchSize = batchSize; + } + + @Override + protected void compute() + { + PriorityQueue> cursors = new PriorityQueue<>(sequences.size()); + for (Sequence s : sequences) { + Yielder> batchYielder = OrderedResultBatch.fromSequence(s, batchSize); + if (!(batchYielder.get() == null || (batchYielder.get().isDrained() && batchYielder.isDone()))) { + YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( + batchYielder, + orderingFn + ); + cursors.offer(batchedSequenceYielder); + } + } + + if (cursors.isEmpty()) { + outputQueue.offer(new OrderedResultBatch<>()); + } else { + invokeAll(new MergeCombineAction( + cursors, + outputQueue, + orderingFn, + combineFn, + null, + yieldAfter, + batchSize + )); + } + } + } + /** * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link BlockingQueue} * in order to construct a set of {@link BlockingQueueuBatchedResultsCursor} to feed to a {@link MergeCombineAction} @@ -516,7 +558,7 @@ protected void compute() * {@link OrderedResultBatch} that are provided by a {@link Yielder} because they will have the initial result batch * available by virtue of translating the sequence. */ - private static class FinalMergeCombineSeedAction extends RecursiveAction + private static class BlockingQueueMergeCombineSeedAction extends RecursiveAction { private final List>> queues; private final Ordering orderingFn; @@ -527,7 +569,7 @@ private static class FinalMergeCombineSeedAction extends RecursiveAction private final boolean hasTimeout; private final long timeoutAt; - private FinalMergeCombineSeedAction( + private BlockingQueueMergeCombineSeedAction( List>> queues, QueuePusher> outputQueue, Ordering orderingFn, @@ -559,9 +601,12 @@ protected void compute() // a yielder starts with the first value outputCursor.initialize(); - cursors.offer(outputCursor); + if (!outputCursor.isDone()) { + cursors.offer(outputCursor); + } } + getPool().execute(new MergeCombineAction( cursors, outputQueue, @@ -869,7 +914,11 @@ static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor< @Override public void initialize() { - nextBatch(); + if (queue.isEmpty()) { + nextBatch(); + } else { + resultBatch = queue.poll(); + } } @Override From 2667e7f037c60c7a33454717e52b6ce91b5b18b6 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 20 Sep 2019 20:58:06 -0700 Subject: [PATCH 06/51] fix --- .../util/common/guava/ParallelMergeCombiningSequence.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index c0276df15d58..c1fe3824c8e2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -332,9 +332,10 @@ void spawnParallelTasks(int parallelMergeTasks) tasks.add(mergeAction); } -// for (RecursiveAction task : tasks) { -// getPool().execute(task); -// } + /* + for (RecursiveAction task : tasks) { + getPool().execute(task); + }*/ invokeAll(tasks); QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); From 94a7ef7d92775cd0b1449066a1ecd43e2c311ec3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 20 Sep 2019 21:21:36 -0700 Subject: [PATCH 07/51] remove comments --- .../druid/benchmark/query/CachingClusteredClientBenchmark.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index f5ba355c0aeb..3436b965df08 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -149,11 +149,9 @@ public class CachingClusteredClientBenchmark public static final ObjectMapper JSON_MAPPER; @Param({"8", "32", "64"}) -// @Param({"8"}) private int numServers; -// @Param({"0", "1", "3", "4", "5", "6", "7", "8"}) @Param({"0", "1", "4"}) private int parallelism; @@ -161,7 +159,6 @@ public class CachingClusteredClientBenchmark private int rowsPerSegment; @Param({"all", "minute"}) -// @Param({"all"}) private String queryGranularity; private QueryToolChestWarehouse toolChestWarehouse; From 754deec7fbe05ac532aae5dd0fcd60eaebfa6264 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 20 Sep 2019 21:48:01 -0700 Subject: [PATCH 08/51] log.info for debug --- .../util/common/guava/ParallelMergeCombiningSequence.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index c1fe3824c8e2..b16b68daeeac 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -250,7 +250,7 @@ protected void compute() // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, // just serially perform the merge-combine with a single task if (sequences.size() < 4 || parallelMergeTasks < 2) { - LOG.debug( + LOG.info( "Input sequence count: %s or available parallel merge task count: %s too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", sequences.size(), @@ -268,7 +268,7 @@ protected void compute() ); invokeAll(mergeAction); } else { - LOG.debug("Spawning %s parallel merge-combine tasks", parallelMergeTasks); + LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelMergeTasks, sequences.size()); spawnParallelTasks(parallelMergeTasks); } } @@ -281,7 +281,7 @@ int computeNumTasks() final int runningThreadCount = getPool().getRunningThreadCount(); final int submissionCount = getPool().getQueuedSubmissionCount(); final long queuedTaskCount = getPool().getQueuedTaskCount(); - LOG.debug( + LOG.info( "processors: [%s] pool parallelism: [%s] active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s]", numProcessors, poolParallelism, @@ -465,7 +465,7 @@ protected void compute() 1L ); final int nextYieldAfter = Math.max(Ints.checkedCast(10 * (yieldAfter / elapsedMillis)), 1); - LOG.debug( + LOG.info( "%s yielded results ran for %s millis, next task yielding every %s operations", yieldAfter, elapsedMillis, From 43f8338a33fb642fa6bb0193ec97a50253e26ab8 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 23 Sep 2019 18:31:14 -0700 Subject: [PATCH 09/51] javadoc --- .../common/guava/ParallelMergeCombiningSequence.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index b16b68daeeac..f43436d35d00 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -332,11 +332,9 @@ void spawnParallelTasks(int parallelMergeTasks) tasks.add(mergeAction); } - /* for (RecursiveAction task : tasks) { getPool().execute(task); - }*/ - invokeAll(tasks); + } QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); BlockingQueueMergeCombineSeedAction finalMergeAction = new BlockingQueueMergeCombineSeedAction<>( @@ -491,9 +489,11 @@ protected void compute() } /** - * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link BlockingQueue} + * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link Yielder} * in order to construct a set of {@link YielderBatchedResultsCursor} to feed to a {@link MergeCombineAction} * which will do the actual work of merging and combining the result batches. + * + * */ private static class YielderMergeCombineSeedAction extends RecursiveAction { @@ -555,9 +555,7 @@ protected void compute() /** * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link BlockingQueue} * in order to construct a set of {@link BlockingQueueuBatchedResultsCursor} to feed to a {@link MergeCombineAction} - * which will do the actual work of merging and combining the result batches. This is not needed for - * {@link OrderedResultBatch} that are provided by a {@link Yielder} because they will have the initial result batch - * available by virtue of translating the sequence. + * which will do the actual work of merging and combining the result batches. */ private static class BlockingQueueMergeCombineSeedAction extends RecursiveAction { From 7e25be74b01e13eeebf03f5ac04eb58d0fd3852f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 25 Sep 2019 03:12:50 -0700 Subject: [PATCH 10/51] safer block for sequence to yielder conversion --- .../CachingClusteredClientBenchmark.java | 2 +- .../guava/ParallelMergeCombiningSequence.java | 49 +++++++++++++++++-- 2 files changed, 47 insertions(+), 4 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 3436b965df08..8cd70b6aff3a 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -148,7 +148,7 @@ public class CachingClusteredClientBenchmark private static final String DATA_SOURCE = "ds"; public static final ObjectMapper JSON_MAPPER; - @Param({"8", "32", "64"}) + @Param({"8", "24"}) private int numServers; diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index f43436d35d00..1d7398eecc8f 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -282,13 +282,14 @@ int computeNumTasks() final int submissionCount = getPool().getQueuedSubmissionCount(); final long queuedTaskCount = getPool().getQueuedTaskCount(); LOG.info( - "processors: [%s] pool parallelism: [%s] active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s]", + "processors: [%s] pool parallelism: [%s] active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s] steal: [%s]", numProcessors, poolParallelism, activeThreadCount, runningThreadCount, submissionCount, - queuedTaskCount + queuedTaskCount, + getPool().getStealCount() ); // max is minimum of either number of processors or user suggested parallelism final int maxParallelism = Math.min(numProcessors, parallelism); @@ -526,7 +527,8 @@ protected void compute() { PriorityQueue> cursors = new PriorityQueue<>(sequences.size()); for (Sequence s : sequences) { - Yielder> batchYielder = OrderedResultBatch.fromSequence(s, batchSize); + final SequenceYielder yielder = new SequenceYielder<>(s, batchSize); + final Yielder> batchYielder = yielder.getYielder(); if (!(batchYielder.get() == null || (batchYielder.get().isDrained() && batchYielder.isDone()))) { YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( batchYielder, @@ -618,6 +620,47 @@ protected void compute() } } + + /** + * {@link ForkJoinPool} friendly {@link Sequence} to {@link OrderedResultBatch} {@link Yielder} + */ + private static class SequenceYielder implements ForkJoinPool.ManagedBlocker + { + private final Sequence sequence; + private final int batchSize; + private volatile Yielder> batchYielder; + + public SequenceYielder(Sequence sequence, int batchSize) + { + this.sequence = sequence; + this.batchSize = batchSize; + } + + public Yielder> getYielder() + { + try { + ForkJoinPool.managedBlock(this); + return batchYielder; + } + catch (InterruptedException e) { + throw new RuntimeException("Failed to load next batch of results", e); + } + } + @Override + public boolean block() throws InterruptedException + { + batchYielder = OrderedResultBatch.fromSequence(sequence, batchSize); + return true; + } + + @Override + public boolean isReleasable() + { + return batchYielder != null; + } + } + + /** * {@link ForkJoinPool} friendly {@link BlockingQueue} feeder, adapted from 'QueueTaker' of Java documentation on * {@link ForkJoinPool.ManagedBlocker}, From 761c6807bbbd84952715a11b0f0b9b85572e4ea6 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 26 Sep 2019 00:54:54 -0700 Subject: [PATCH 11/51] refactor LifecycleForkJoinPool into LifecycleForkJoinPoolProvider which wraps a ForkJoinPool --- .../CachingClusteredClientBenchmark.java | 12 +++++++---- .../movingaverage/MovingAverageQueryTest.java | 3 +-- ...ava => LifecycleForkJoinPoolProvider.java} | 20 ++++++++++++------- .../druid/client/CachingClusteredClient.java | 7 ++++--- .../druid/guice/DruidProcessingModule.java | 11 ++++++++-- ...chingClusteredClientFunctionalityTest.java | 3 +-- .../client/CachingClusteredClientTest.java | 9 +-------- 7 files changed, 37 insertions(+), 28 deletions(-) rename processing/src/main/java/org/apache/druid/guice/{LifecycleForkJoinPool.java => LifecycleForkJoinPoolProvider.java} (79%) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 8cd70b6aff3a..35d40442c5cc 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -47,7 +47,6 @@ import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.StupidPool; -import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.concurrent.Execs; @@ -165,7 +164,7 @@ public class CachingClusteredClientBenchmark private QueryRunnerFactoryConglomerate conglomerate; private CachingClusteredClient cachingClusteredClient; private ExecutorService processingPool; - private LifecycleForkJoinPool forkJoinPool; + private ForkJoinPool forkJoinPool; private boolean parallelCombine; @@ -306,7 +305,12 @@ public > QueryToolChest getToolChest } processingPool = Execs.multiThreaded(processingConfig.getNumThreads(), "caching-clustered-client-benchmark"); - forkJoinPool = new LifecycleForkJoinPool((int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true, 1000L); + forkJoinPool = new ForkJoinPool( + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + ForkJoinPool.defaultForkJoinWorkerThreadFactory, + null, + true + ); cachingClusteredClient = new CachingClusteredClient( toolChestWarehouse, serverView, @@ -369,7 +373,7 @@ public void tearDown() throws IOException { closer.close(); processingPool.shutdown(); - forkJoinPool.stop(); + forkJoinPool.shutdownNow(); } @Benchmark diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index fadd572eb361..05d3b2352382 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -41,7 +41,6 @@ import org.apache.druid.data.input.Row; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.http.DruidHttpClientConfig; @@ -352,7 +351,7 @@ public long getMaxQueuedBytes() return 0L; } }, - new LifecycleForkJoinPool(6, ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true, 1000L) + ForkJoinPool.commonPool() ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPoolProvider.java similarity index 79% rename from processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java rename to processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPoolProvider.java index cf6dcca9ed53..afae29d6bcb4 100644 --- a/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPool.java +++ b/processing/src/main/java/org/apache/druid/guice/LifecycleForkJoinPoolProvider.java @@ -26,20 +26,21 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; -public class LifecycleForkJoinPool extends ForkJoinPool +public class LifecycleForkJoinPoolProvider { - private static final Logger LOG = new Logger(LifecycleForkJoinPool.class); + private static final Logger LOG = new Logger(LifecycleForkJoinPoolProvider.class); private final long awaitShutdownMillis; + private final ForkJoinPool pool; - public LifecycleForkJoinPool( + public LifecycleForkJoinPoolProvider( int parallelism, - ForkJoinWorkerThreadFactory factory, + ForkJoinPool.ForkJoinWorkerThreadFactory factory, Thread.UncaughtExceptionHandler handler, boolean asyncMode, long awaitShutdownMillis ) { - super(parallelism, factory, handler, asyncMode); + this.pool = new ForkJoinPool(parallelism, factory, handler, asyncMode); this.awaitShutdownMillis = awaitShutdownMillis; } @@ -47,9 +48,9 @@ public LifecycleForkJoinPool( public void stop() { LOG.info("Shutting down ForkJoinPool [%s]", this); - shutdown(); + pool.shutdown(); try { - if (!awaitTermination(awaitShutdownMillis, TimeUnit.MILLISECONDS)) { + if (!pool.awaitTermination(awaitShutdownMillis, TimeUnit.MILLISECONDS)) { LOG.warn("Failed to complete all tasks in FJP [%s]", this); } } @@ -58,4 +59,9 @@ public void stop() throw new RuntimeException("interrupted on shutdown", e); } } + + public ForkJoinPool getPool() + { + return pool; + } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 971276d7fee5..2c814cf599bb 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -36,8 +36,8 @@ import org.apache.druid.client.cache.CachePopulator; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.annotations.Client; +import org.apache.druid.guice.annotations.Merging; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.Intervals; @@ -89,6 +89,7 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; +import java.util.concurrent.ForkJoinPool; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -104,7 +105,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final CachePopulator cachePopulator; private final CacheConfig cacheConfig; private final DruidHttpClientConfig httpClientConfig; - private final LifecycleForkJoinPool pool; + private final ForkJoinPool pool; @Inject public CachingClusteredClient( @@ -115,7 +116,7 @@ public CachingClusteredClient( CachePopulator cachePopulator, CacheConfig cacheConfig, @Client DruidHttpClientConfig httpClientConfig, - LifecycleForkJoinPool pool + @Merging ForkJoinPool pool ) { this.warehouse = warehouse; diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index a9eee92b34d2..fcc5026e9deb 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -138,9 +138,9 @@ public BlockingPool getMergeBufferPool(DruidProcessingConfig config) @Provides @ManageLifecycle - public LifecycleForkJoinPool getMergeProcessingPool(DruidProcessingConfig config) + public LifecycleForkJoinPoolProvider getMergeProcessingPoolProvider(DruidProcessingConfig config) { - return new LifecycleForkJoinPool( + return new LifecycleForkJoinPoolProvider( config.getNumThreadsMergePool(), ForkJoinPool.defaultForkJoinWorkerThreadFactory, // todo: ? null, @@ -149,6 +149,13 @@ public LifecycleForkJoinPool getMergeProcessingPool(DruidProcessingConfig config ); } + @Provides + @Merging + public ForkJoinPool getMergeProcessingPool(LifecycleForkJoinPoolProvider poolProvider) + { + return poolProvider.getPool(); + } + private void verifyDirectMemory(DruidProcessingConfig config) { try { diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index be7ede0c2cfa..a915bed0443a 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -35,7 +35,6 @@ import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.client.selector.TierSelectorStrategy; -import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -316,7 +315,7 @@ public long getMaxQueuedBytes() return 0L; } }, - new LifecycleForkJoinPool(6, ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true, 1000L) + ForkJoinPool.commonPool() ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 72eae6719ca7..d8b0d2774e23 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -52,7 +52,6 @@ import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.RandomServerSelectorStrategy; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.guice.LifecycleForkJoinPool; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; @@ -2759,13 +2758,7 @@ public long getMaxQueuedBytes() return 0L; } }, - new LifecycleForkJoinPool( - (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), - ForkJoinPool.defaultForkJoinWorkerThreadFactory, - null, - true, - 1000L - ) + ForkJoinPool.commonPool() ); } From e8cc5a8d5df60c5f7aade4dac42163048cd80315 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 26 Sep 2019 18:03:52 -0700 Subject: [PATCH 12/51] smooth yield rate adjustment, more logs to help tune --- .../guava/ParallelMergeCombiningSequence.java | 37 +++++++++++++------ .../ParallelMergeCombiningSequenceTest.java | 4 +- 2 files changed, 28 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 1d7398eecc8f..d12312503083 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -293,8 +293,8 @@ int computeNumTasks() ); // max is minimum of either number of processors or user suggested parallelism final int maxParallelism = Math.min(numProcessors, parallelism); - // adjust max to be no more than total pool parallelism less the number of active threads - final int computedParallelism = Math.min(maxParallelism, poolParallelism - activeThreadCount); + // adjust max to be no more than total pool parallelism less the number of running threads + final int computedParallelism = Math.min(maxParallelism, poolParallelism - runningThreadCount); // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot, we need at least 2 final int computedOptimalParallelism = Math.min((int) Math.floor((double) sequences.size() / 2.0), computedParallelism - 1); return computedOptimalParallelism; @@ -377,6 +377,7 @@ private static class MergeCombineAction extends RecursiveAction private final T initialValue; private final int yieldAfter; private final int batchSize; + private final int depth; private MergeCombineAction( PriorityQueue> pQueue, @@ -385,7 +386,8 @@ private MergeCombineAction( BinaryOperator combineFn, T initialValue, int yieldAfter, - int batchSize + int batchSize, + int depth ) { this.pQueue = pQueue; @@ -395,6 +397,7 @@ private MergeCombineAction( this.initialValue = initialValue; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.depth = depth; } @Override @@ -463,12 +466,21 @@ protected void compute() TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS), 1L ); - final int nextYieldAfter = Math.max(Ints.checkedCast(10 * (yieldAfter / elapsedMillis)), 1); + final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); + final double cumulativeMovingAverage = (nextYieldAfter + (depth * yieldAfter)) / (depth + 1); + final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); + LOG.info( - "%s yielded results ran for %s millis, next task yielding every %s operations", + "stage %s processed %s results in %s millis, next task yielding after %s operations (active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s] steal: [%s])", + depth, yieldAfter, elapsedMillis, - nextYieldAfter + adjustedNextYieldAfter, + getPool().getActiveThreadCount(), + getPool().getRunningThreadCount(), + getPool().getQueuedSubmissionCount(), + getPool().getQueuedTaskCount(), + getPool().getStealCount() ); getPool().execute(new MergeCombineAction<>( pQueue, @@ -476,8 +488,9 @@ protected void compute() orderingFn, combineFn, currentCombinedValue, - nextYieldAfter, - batchSize + adjustedNextYieldAfter, + batchSize, + depth + 1 )); } else { // if priority queue is empty, push the final accumulated value @@ -548,7 +561,8 @@ protected void compute() combineFn, null, yieldAfter, - batchSize + batchSize, + 1 )); } } @@ -615,7 +629,8 @@ protected void compute() combineFn, null, yieldAfter, - batchSize + batchSize, + 1 )); } } @@ -647,7 +662,7 @@ public Yielder> getYielder() } } @Override - public boolean block() throws InterruptedException + public boolean block() { batchYielder = OrderedResultBatch.fromSequence(sequence, batchSize); return true; diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index e04aee43b049..97e58156ccb1 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -381,8 +381,8 @@ private void assertResult(List> sequences) throws InterruptedE 5000, 0, Runtime.getRuntime().availableProcessors() - 1, - 8, - 4 + 4, + 2 ); Yielder combiningYielder = Yielders.each(combiningSequence); From bcc90a466d217b45c24256705fa8f06646a8667d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 27 Sep 2019 04:11:09 -0700 Subject: [PATCH 13/51] cleanup, less logs --- .../guava/ParallelMergeCombiningSequence.java | 140 +++++++----------- 1 file changed, 50 insertions(+), 90 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index d12312503083..3be52f2ed133 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -92,41 +92,39 @@ public ParallelMergeCombiningSequence( @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); - - if (baseSequences.size() > 0) { - // 2 layer parallel merge done in fjp - ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( - baseSequences, - orderingFn, - combineFn, - outputQueue, - queueSize, - parallelism, - yieldAfter, - batchSize, - hasTimeout, - timeoutAt - ); - workerPool.execute(finalMergeAction); - Sequence finalOutSequence = makeOutputSequenceForQueue(finalMergeAction, outputQueue, hasTimeout, timeoutAt); - return finalOutSequence.toYielder(initValue, accumulator); + if (baseSequences.isEmpty()) { + return Sequences.empty().toYielder(initValue, accumulator); } - // empty result - return Sequences.empty().toYielder(initValue, accumulator); + + final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); + ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( + baseSequences, + orderingFn, + combineFn, + outputQueue, + queueSize, + parallelism, + yieldAfter, + batchSize, + hasTimeout, + timeoutAt + ); + workerPool.execute(finalMergeAction); + Sequence finalOutSequence = makeOutputSequenceForQueue(outputQueue, hasTimeout, timeoutAt); + return finalOutSequence.toYielder(initValue, accumulator); } /** * Create an output {@link Sequence} that wraps the output {@link BlockingQueue} of a {@link RecursiveAction} task */ private static Sequence makeOutputSequenceForQueue( - RecursiveAction task, BlockingQueue> queue, boolean hasTimeout, long timeoutAt ) { - final Sequence backgroundCombineSequence = new BaseSequence<>( + + return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override @@ -179,22 +177,10 @@ public T next() @Override public void cleanup(Iterator iterFromMake) { - try { - // todo: hmm.. this probably does nothing since this is only the first task and it doesn't run until all - // recursive tasks that are spawned complete, maybe we need a collection of 'active' tasks which the tasks - // themselves update? - if (!task.isDone()) { - task.cancel(true); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } + // todo: do ... something? } } ); - - return backgroundCombineSequence; } /** @@ -251,7 +237,7 @@ protected void compute() // just serially perform the merge-combine with a single task if (sequences.size() < 4 || parallelMergeTasks < 2) { LOG.info( - "Input sequence count: %s or available parallel merge task count: %s too small to perform parallel" + "Input sequence count (%s) or available parallel merge task count (%s) too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", sequences.size(), parallelMergeTasks @@ -266,47 +252,14 @@ protected void compute() yieldAfter, batchSize ); - invokeAll(mergeAction); + getPool().execute(mergeAction); } else { + // 2 layer parallel merge done in fjp LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelMergeTasks, sequences.size()); spawnParallelTasks(parallelMergeTasks); } } - int computeNumTasks() - { - final int numProcessors = JvmUtils.getRuntimeInfo().getAvailableProcessors(); - final int poolParallelism = getPool().getParallelism(); - final int activeThreadCount = getPool().getActiveThreadCount(); - final int runningThreadCount = getPool().getRunningThreadCount(); - final int submissionCount = getPool().getQueuedSubmissionCount(); - final long queuedTaskCount = getPool().getQueuedTaskCount(); - LOG.info( - "processors: [%s] pool parallelism: [%s] active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s] steal: [%s]", - numProcessors, - poolParallelism, - activeThreadCount, - runningThreadCount, - submissionCount, - queuedTaskCount, - getPool().getStealCount() - ); - // max is minimum of either number of processors or user suggested parallelism - final int maxParallelism = Math.min(numProcessors, parallelism); - // adjust max to be no more than total pool parallelism less the number of running threads - final int computedParallelism = Math.min(maxParallelism, poolParallelism - runningThreadCount); - // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot, we need at least 2 - final int computedOptimalParallelism = Math.min((int) Math.floor((double) sequences.size() / 2.0), computedParallelism - 1); - return computedOptimalParallelism; - - - /* original manual control - final int suggestedParallelism = Math.max(2, parallelism - 1); - final int suggestedOptimalParallelism = Math.min((int) Math.floor((double) yielders.size() / 2.0), suggestedParallelism); - return suggestedOptimalParallelism; - */ - } - void spawnParallelTasks(int parallelMergeTasks) { List tasks = new ArrayList<>(); @@ -351,6 +304,27 @@ void spawnParallelTasks(int parallelMergeTasks) getPool().execute(finalMergeAction); } + + /** + * Computes maximum number of layer 1 parallel merging tasks given available processors and an estimate of current + * {@link ForkJoinPool} utilization. A return value of 1 or less indicates that a serial merge will be done on + * the pool instead. + */ + int computeNumTasks() + { + // max is minimum of either number of processors or user suggested parallelism + final int maxParallelism = Math.min(JvmUtils.getRuntimeInfo().getAvailableProcessors(), parallelism); + // adjust max to be no more than total pool parallelism less the number of running threads + submitted tasks + final int utilizationEstimate = getPool().getRunningThreadCount() + getPool().getQueuedSubmissionCount(); + // minimum of 'max computed parallelism' and pool parallelism less current 'utilization estimate' + final int computedParallelism = Math.min(maxParallelism, getPool().getParallelism() - utilizationEstimate); + // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot + final int computedOptimalParallelism = Math.min( + (int) Math.floor((double) sequences.size() / 2.0), + computedParallelism - 1 + ); + return computedOptimalParallelism; + } } @@ -469,19 +443,6 @@ protected void compute() final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); final double cumulativeMovingAverage = (nextYieldAfter + (depth * yieldAfter)) / (depth + 1); final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); - - LOG.info( - "stage %s processed %s results in %s millis, next task yielding after %s operations (active thread count: [%s] running thread count: [%s] submission count: [%s] queued task count: [%s] steal: [%s])", - depth, - yieldAfter, - elapsedMillis, - adjustedNextYieldAfter, - getPool().getActiveThreadCount(), - getPool().getRunningThreadCount(), - getPool().getQueuedSubmissionCount(), - getPool().getQueuedTaskCount(), - getPool().getStealCount() - ); getPool().execute(new MergeCombineAction<>( pQueue, outputQueue, @@ -554,7 +515,7 @@ protected void compute() if (cursors.isEmpty()) { outputQueue.offer(new OrderedResultBatch<>()); } else { - invokeAll(new MergeCombineAction( + getPool().execute(new MergeCombineAction( cursors, outputQueue, orderingFn, @@ -612,9 +573,8 @@ protected void compute() for (BlockingQueue> queue : queues) { BatchedResultsCursor outputCursor = new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt); - // seed the Drainer before we can add to pQueue, this is because a blocking queue starts empty where - // a yielder starts with the first value + // this is blocking outputCursor.initialize(); if (!outputCursor.isDone()) { cursors.offer(outputCursor); @@ -658,7 +618,7 @@ public Yielder> getYielder() return batchYielder; } catch (InterruptedException e) { - throw new RuntimeException("Failed to load next batch of results", e); + throw new RuntimeException("Failed to load initial batch of results", e); } } @Override @@ -800,7 +760,7 @@ static Yielder> fromSequence(Sequence sequence, int public OrderedResultBatch accumulate(OrderedResultBatch accumulated, E in) { count++; - if (/*count == 1 ||*/ count % batchSize == 0) { + if (count % batchSize == 0) { yield(); } accumulated.add(in); From 41f09f9f380c6694101c4b88f188d8ab46be3988 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 1 Oct 2019 05:41:58 -0700 Subject: [PATCH 14/51] error handling, bug fixes, on by default, more parallel, more tests --- .../CachingClusteredClientBenchmark.java | 7 + .../org/apache/druid/java/util/common/RE.java | 5 + .../guava/ParallelMergeCombiningSequence.java | 433 ++++++---- .../ParallelMergeCombiningSequenceTest.java | 760 ++++++++++++------ .../MaterializedViewQueryQueryToolChest.java | 17 + .../movingaverage/MovingAverageQueryTest.java | 9 + .../druid/query/DruidProcessingConfig.java | 6 + .../org/apache/druid/query/QueryContexts.java | 4 +- .../apache/druid/query/QueryToolChest.java | 3 +- .../groupby/strategy/GroupByStrategy.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 28 +- .../druid/client/CachingClusteredClient.java | 11 +- ...chingClusteredClientFunctionalityTest.java | 12 +- .../client/CachingClusteredClientTest.java | 9 + 14 files changed, 885 insertions(+), 421 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 35d40442c5cc..6e4768946570 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -239,6 +239,12 @@ public int getNumThreads() { return numProcessingThreads; } + + @Override + public boolean useParallelMergePool() + { + return true; + } }; conglomerate = new DefaultQueryRunnerFactoryConglomerate( @@ -319,6 +325,7 @@ public > QueryToolChest getToolChest new ForegroundCachePopulator(JSON_MAPPER, new CachePopulatorStats(), 0), new CacheConfig(), new DruidHttpClientConfig(), + processingConfig, forkJoinPool ); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/RE.java b/core/src/main/java/org/apache/druid/java/util/common/RE.java index f7a52a65c0b1..8a6fa4073330 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/RE.java +++ b/core/src/main/java/org/apache/druid/java/util/common/RE.java @@ -32,4 +32,9 @@ public RE(Throwable cause, String formatText, Object... arguments) { super(StringUtils.nonStrictFormat(formatText, arguments), cause); } + + public RE(Throwable cause) + { + super(cause == null ? null : cause.getMessage(), cause); + } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 3be52f2ed133..c08e04c6708c 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.utils.JvmUtils; @@ -61,15 +62,15 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final int yieldAfter; private final int batchSize; private final int parallelism; + private final CancellationGizmo cancellationGizmo; public ParallelMergeCombiningSequence( ForkJoinPool workerPool, List> baseSequences, Ordering orderingFn, BinaryOperator combineFn, - int queueSize, boolean hasTimeout, - long timeout, + long timeoutMillis, int queryPriority, int parallelism, int yieldAfter, @@ -80,13 +81,14 @@ public ParallelMergeCombiningSequence( this.baseSequences = baseSequences; this.orderingFn = orderingFn; this.combineFn = combineFn; - this.queueSize = queueSize; this.hasTimeout = hasTimeout; - this.timeoutAt = System.currentTimeMillis() + timeout; + this.timeoutAt = System.currentTimeMillis() + timeoutMillis; this.queryPriority = queryPriority; this.parallelism = parallelism; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.queueSize = 4 * (yieldAfter / batchSize); + this.cancellationGizmo = new CancellationGizmo(); } @Override @@ -107,20 +109,22 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat yieldAfter, batchSize, hasTimeout, - timeoutAt + timeoutAt, + cancellationGizmo ); workerPool.execute(finalMergeAction); - Sequence finalOutSequence = makeOutputSequenceForQueue(outputQueue, hasTimeout, timeoutAt); + Sequence finalOutSequence = makeOutputSequenceForQueue(outputQueue, hasTimeout, timeoutAt, cancellationGizmo); return finalOutSequence.toYielder(initValue, accumulator); } /** * Create an output {@link Sequence} that wraps the output {@link BlockingQueue} of a {@link RecursiveAction} task */ - private static Sequence makeOutputSequenceForQueue( + static Sequence makeOutputSequenceForQueue( BlockingQueue> queue, boolean hasTimeout, - long timeoutAt + long timeoutAt, + CancellationGizmo cancellationGizmo ) { @@ -137,35 +141,55 @@ public Iterator make() @Override public boolean hasNext() { + final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); + if (thisTimeout < 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) { - final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); currentBatch = queue.poll(thisTimeout, TimeUnit.MILLISECONDS); } else { currentBatch = queue.take(); } } - if (currentBatch == null) { - throw new RuntimeException(new TimeoutException()); + throw new RE(new TimeoutException("Sequence iterator timed out waiting for data")); + } + + if (cancellationGizmo.isCancelled()) { + if (cancellationGizmo.getRuntimeException() != null) { + throw cancellationGizmo.getRuntimeException(); + } else { + throw new RuntimeException("Failed to merge results, unknown error"); + } } + if (currentBatch.isTerminalResult()) { return false; } return true; } catch (InterruptedException e) { - throw new RuntimeException(e); + throw new RE(e); } } @Override public T next() { + if (cancellationGizmo.isCancelled()) { + if (cancellationGizmo.getRuntimeException() != null) { + throw cancellationGizmo.getRuntimeException(); + } else { + throw new RuntimeException("Failed to merge results, unknown error"); + } + } + if (currentBatch == null || currentBatch.isDrained() || currentBatch.isTerminalResult()) { throw new NoSuchElementException(); } @@ -202,6 +226,7 @@ private static class ParallelMergeCombineAction extends RecursiveAction private final int batchSize; private final boolean hasTimeout; private final long timeoutAt; + private final CancellationGizmo cancellationGizmo; private ParallelMergeCombineAction( List> sequences, @@ -213,7 +238,8 @@ private ParallelMergeCombineAction( int yieldAfter, int batchSize, boolean hasTimeout, - long timeoutAt + long timeoutAt, + CancellationGizmo cancellationGizmo ) { this.sequences = sequences; @@ -226,37 +252,45 @@ private ParallelMergeCombineAction( this.batchSize = batchSize; this.hasTimeout = hasTimeout; this.timeoutAt = timeoutAt; + this.cancellationGizmo = cancellationGizmo; } @Override protected void compute() { - final int parallelMergeTasks = computeNumTasks(); - - // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, - // just serially perform the merge-combine with a single task - if (sequences.size() < 4 || parallelMergeTasks < 2) { - LOG.info( - "Input sequence count (%s) or available parallel merge task count (%s) too small to perform parallel" - + " merge-combine, performing serially with a single merge-combine task", - sequences.size(), - parallelMergeTasks - ); + try { + final int parallelMergeTasks = computeNumTasks(); + + // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, + // just serially perform the merge-combine with a single task + if (sequences.size() < 4 || parallelMergeTasks < 2) { + LOG.info( + "Input sequence count (%s) or available parallel merge task count (%s) too small to perform parallel" + + " merge-combine, performing serially with a single merge-combine task", + sequences.size(), + parallelMergeTasks + ); - QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); - YielderMergeCombineSeedAction mergeAction = new YielderMergeCombineSeedAction<>( - sequences, - resultsPusher, - orderingFn, - combineFn, - yieldAfter, - batchSize - ); - getPool().execute(mergeAction); - } else { - // 2 layer parallel merge done in fjp - LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelMergeTasks, sequences.size()); - spawnParallelTasks(parallelMergeTasks); + QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + YielderMergeCombineSeedAction mergeAction = new YielderMergeCombineSeedAction<>( + sequences, + resultsPusher, + orderingFn, + combineFn, + yieldAfter, + batchSize, + cancellationGizmo + ); + getPool().execute(mergeAction); + } else { + // 2 layer parallel merge done in fjp + LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelMergeTasks, sequences.size()); + spawnParallelTasks(parallelMergeTasks); + } + } + catch (Exception ex) { + cancellationGizmo.cancel(ex); + out.offer(new OrderedResultBatch<>()); } } @@ -281,7 +315,8 @@ void spawnParallelTasks(int parallelMergeTasks) orderingFn, combineFn, yieldAfter, - batchSize + batchSize, + cancellationGizmo ); tasks.add(mergeAction); } @@ -299,7 +334,8 @@ void spawnParallelTasks(int parallelMergeTasks) yieldAfter, batchSize, hasTimeout, - timeoutAt + timeoutAt, + cancellationGizmo ); getPool().execute(finalMergeAction); @@ -352,6 +388,7 @@ private static class MergeCombineAction extends RecursiveAction private final int yieldAfter; private final int batchSize; private final int depth; + private final CancellationGizmo cancellationGizmo; private MergeCombineAction( PriorityQueue> pQueue, @@ -361,7 +398,8 @@ private MergeCombineAction( T initialValue, int yieldAfter, int batchSize, - int depth + int depth, + CancellationGizmo cancellationGizmo ) { this.pQueue = pQueue; @@ -372,92 +410,104 @@ private MergeCombineAction( this.yieldAfter = yieldAfter; this.batchSize = batchSize; this.depth = depth; + this.cancellationGizmo = cancellationGizmo; } @Override protected void compute() { - long start = System.nanoTime(); + try { + long start = System.nanoTime(); - int counter = 0; - int batchCounter = 0; - OrderedResultBatch outputBatch = new OrderedResultBatch<>(batchSize); + int counter = 0; + int batchCounter = 0; + OrderedResultBatch outputBatch = new OrderedResultBatch<>(batchSize); + OrderedResultBatch errorBatch = null; - T currentCombinedValue = initialValue; - while (counter++ < yieldAfter && !pQueue.isEmpty()) { + T currentCombinedValue = initialValue; + while (counter++ < yieldAfter && !pQueue.isEmpty()) { + BatchedResultsCursor cursor = pQueue.poll(); - BatchedResultsCursor cursor = pQueue.poll(); - // get the next value to accumulate + // push the queue along + if (!cursor.isDone()) { + T nextValueToAccumulate = cursor.get(); - // push the queue along - if (!cursor.isDone()) { - T nextValueToAccumulate = cursor.get(); + cursor.advance(); + if (!cursor.isDone()) { + pQueue.offer(cursor); + } else { + cursor.close(); + } - cursor.advance(); - if (!cursor.isDone()) { - pQueue.offer(cursor); - } else { - cursor.close(); - } + // if current value is null, combine null with next value + if (currentCombinedValue == null) { + currentCombinedValue = combineFn.apply(null, nextValueToAccumulate); + continue; + } - // if current value is null, combine null with next value - if (currentCombinedValue == null) { - currentCombinedValue = combineFn.apply(null, nextValueToAccumulate); - continue; - } + // if current value is "same" as next value, combine them + if (orderingFn.compare(currentCombinedValue, nextValueToAccumulate) == 0) { + currentCombinedValue = combineFn.apply(currentCombinedValue, nextValueToAccumulate); + continue; + } - // if current value is "same" as next value, combine them - if (orderingFn.compare(currentCombinedValue, nextValueToAccumulate) == 0) { - currentCombinedValue = combineFn.apply(currentCombinedValue, nextValueToAccumulate); - continue; - } + // else, push accumulated value to the queue, accumulate again with next value as initial - // else, push accumulated value to the queue, accumulate again with next value as initial - if (batchCounter < batchSize) { outputBatch.add(currentCombinedValue); batchCounter++; + if (batchCounter >= batchSize) { + outputQueue.offer(outputBatch); + outputBatch = new OrderedResultBatch<>(batchSize); + batchCounter = 0; + } + + // next value is now current value + currentCombinedValue = combineFn.apply(null, nextValueToAccumulate); } else { - outputQueue.offer(outputBatch); - outputBatch = new OrderedResultBatch<>(batchSize); - outputBatch.add(currentCombinedValue); - batchCounter = 1; + cursor.close(); } - - // next value is now current value - currentCombinedValue = combineFn.apply(null, nextValueToAccumulate); - } else { - cursor.close(); } - } - if (!pQueue.isEmpty()) { - // if there is still work to be done, execute a new task with the current accumulated value to continue - // combining where we left off - if (!outputBatch.isDrained()) { + if (!pQueue.isEmpty() && !cancellationGizmo.isCancelled()) { + // if there is still work to be done, execute a new task with the current accumulated value to continue + // combining where we left off + if (!outputBatch.isDrained()) { + outputQueue.offer(outputBatch); + } + final long elapsedMillis = Math.max( + TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS), + 1L + ); + final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); + final double cumulativeMovingAverage = (nextYieldAfter + (depth * yieldAfter)) / (depth + 1); + final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); + getPool().execute(new MergeCombineAction<>( + pQueue, + outputQueue, + orderingFn, + combineFn, + currentCombinedValue, + adjustedNextYieldAfter, + batchSize, + depth + 1, + cancellationGizmo + )); + } else if (cancellationGizmo.isCancelled() || errorBatch != null) { + if (errorBatch == null) { + outputQueue.offer(new OrderedResultBatch<>()); + } else { + outputQueue.offer(errorBatch); + } + } else { + // if priority queue is empty, push the final accumulated value + outputBatch.add(currentCombinedValue); outputQueue.offer(outputBatch); + // ... and the terminal value to indicate the blocking queue holding the values is complete + outputQueue.offer(new OrderedResultBatch<>()); } - final long elapsedMillis = Math.max( - TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS), - 1L - ); - final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); - final double cumulativeMovingAverage = (nextYieldAfter + (depth * yieldAfter)) / (depth + 1); - final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); - getPool().execute(new MergeCombineAction<>( - pQueue, - outputQueue, - orderingFn, - combineFn, - currentCombinedValue, - adjustedNextYieldAfter, - batchSize, - depth + 1 - )); - } else { - // if priority queue is empty, push the final accumulated value - outputBatch.add(currentCombinedValue); - outputQueue.offer(outputBatch); - // ... and the terminal value to indicate the blocking queue holding the values is complete + } + catch (Exception ex) { + cancellationGizmo.cancel(ex); outputQueue.offer(new OrderedResultBatch<>()); } } @@ -467,8 +517,6 @@ protected void compute() * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link Yielder} * in order to construct a set of {@link YielderBatchedResultsCursor} to feed to a {@link MergeCombineAction} * which will do the actual work of merging and combining the result batches. - * - * */ private static class YielderMergeCombineSeedAction extends RecursiveAction { @@ -478,6 +526,7 @@ private static class YielderMergeCombineSeedAction extends RecursiveAction private final QueuePusher> outputQueue; private final int yieldAfter; private final int batchSize; + private final CancellationGizmo cancellationGizmo; private YielderMergeCombineSeedAction( List> sequences, @@ -485,7 +534,8 @@ private YielderMergeCombineSeedAction( Ordering orderingFn, BinaryOperator combineFn, int yieldAfter, - int batchSize + int batchSize, + CancellationGizmo cancellationGizmo ) { this.sequences = sequences; @@ -494,37 +544,45 @@ private YielderMergeCombineSeedAction( this.outputQueue = outputQueue; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.cancellationGizmo = cancellationGizmo; } @Override protected void compute() { - PriorityQueue> cursors = new PriorityQueue<>(sequences.size()); - for (Sequence s : sequences) { - final SequenceYielder yielder = new SequenceYielder<>(s, batchSize); - final Yielder> batchYielder = yielder.getYielder(); - if (!(batchYielder.get() == null || (batchYielder.get().isDrained() && batchYielder.isDone()))) { - YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( - batchYielder, - orderingFn - ); - cursors.offer(batchedSequenceYielder); + try { + PriorityQueue> cursors = new PriorityQueue<>(sequences.size()); + for (Sequence s : sequences) { + final SequenceYielder yielder = new SequenceYielder<>(s, batchSize); + final Yielder> batchYielder = yielder.getYielder(); + if (!(batchYielder.get() == null || (batchYielder.get().isDrained() && batchYielder.isDone()))) { + YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( + batchYielder, + orderingFn + ); + cursors.offer(batchedSequenceYielder); + } } - } - if (cursors.isEmpty()) { + if (cursors.isEmpty()) { + outputQueue.offer(new OrderedResultBatch<>()); + } else { + getPool().execute(new MergeCombineAction( + cursors, + outputQueue, + orderingFn, + combineFn, + null, + yieldAfter, + batchSize, + 1, + cancellationGizmo + )); + } + } + catch (Exception ex) { + cancellationGizmo.cancel(ex); outputQueue.offer(new OrderedResultBatch<>()); - } else { - getPool().execute(new MergeCombineAction( - cursors, - outputQueue, - orderingFn, - combineFn, - null, - yieldAfter, - batchSize, - 1 - )); } } } @@ -544,6 +602,7 @@ private static class BlockingQueueMergeCombineSeedAction extends RecursiveAct private final int batchSize; private final boolean hasTimeout; private final long timeoutAt; + private final CancellationGizmo cancellationGizmo; private BlockingQueueMergeCombineSeedAction( List>> queues, @@ -553,7 +612,8 @@ private BlockingQueueMergeCombineSeedAction( int yieldAfter, int batchSize, boolean hasTimeout, - long timeoutAt + long timeoutAt, + CancellationGizmo cancellationGizmo ) { this.queues = queues; @@ -564,37 +624,81 @@ private BlockingQueueMergeCombineSeedAction( this.batchSize = batchSize; this.hasTimeout = hasTimeout; this.timeoutAt = timeoutAt; + this.cancellationGizmo = cancellationGizmo; } @Override protected void compute() { - PriorityQueue> cursors = new PriorityQueue<>(queues.size()); - for (BlockingQueue> queue : queues) { - BatchedResultsCursor outputCursor = - new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt); + try { + PriorityQueue> cursors = new PriorityQueue<>(queues.size()); + for (BlockingQueue> queue : queues) { + BatchedResultsCursor outputCursor = + new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt); + + // this is blocking + outputCursor.initialize(); + if (!outputCursor.isDone()) { + cursors.offer(outputCursor); + } + } - // this is blocking - outputCursor.initialize(); - if (!outputCursor.isDone()) { - cursors.offer(outputCursor); + if (cursors.size() > 0) { + getPool().execute(new MergeCombineAction( + cursors, + outputQueue, + orderingFn, + combineFn, + null, + yieldAfter, + batchSize, + 1, + cancellationGizmo + )); + } else { + outputQueue.offer(new OrderedResultBatch<>()); } } + catch (Exception ex) { + cancellationGizmo.cancel(ex); + outputQueue.offer(new OrderedResultBatch<>()); + } + } + } - getPool().execute(new MergeCombineAction( - cursors, - outputQueue, - orderingFn, - combineFn, - null, - yieldAfter, - batchSize, - 1 - )); + /** + * Token to allow any {@link RecursiveAction} signal the others and the output sequence that something bad happened + * and processing should cancel, such as a timeout or connection loss. + */ + static class CancellationGizmo + { + // volatile instead of AtomicBoolean because it is never unset + private volatile boolean cancelled; + private volatile Exception exception; + + void cancel(Exception ex) + { + if (cancelled) { + return; + } + cancelled = true; + exception = ex; } - } + boolean isCancelled() + { + return cancelled; + } + + RuntimeException getRuntimeException() + { + if (exception instanceof RuntimeException) { + return (RuntimeException) exception; + } + return new RE(exception); + } + } /** * {@link ForkJoinPool} friendly {@link Sequence} to {@link OrderedResultBatch} {@link Yielder} @@ -658,16 +762,22 @@ static class QueuePusher implements ForkJoinPool.ManagedBlocker @Override public boolean block() throws InterruptedException { + boolean success = false; if (item != null) { if (hasTimeout) { final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); - queue.offer(item, thisTimeout, TimeUnit.MILLISECONDS); + if (thisTimeout < 0) { + throw new RE(new TimeoutException("QueuePusher timed out offering data")); + } + success = queue.offer(item, thisTimeout, TimeUnit.MILLISECONDS); } else { - queue.offer(item); + success = queue.offer(item); + } + if (success) { + item = null; } - item = null; } - return true; + return success; } @Override @@ -700,19 +810,19 @@ public void offer(E item) * blocking that is needed to move results between stages of {@link MergeCombineAction} done in parallel, allowing * the fork join tasks to focus on doing actual work instead of dealing with managed blocking. */ - private static class OrderedResultBatch + static class OrderedResultBatch { @Nullable private final Queue values; private final boolean isTerminal; - private OrderedResultBatch(int batchSize) + OrderedResultBatch(int batchSize) { this.values = new ArrayDeque<>(batchSize); this.isTerminal = false; } - private OrderedResultBatch() + OrderedResultBatch() { this.values = null; this.isTerminal = true; @@ -963,12 +1073,15 @@ public boolean block() throws InterruptedException if (resultBatch == null || resultBatch.isDrained()) { if (hasTimeout) { final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); + if (thisTimeout < 0) { + throw new RE(new TimeoutException("BlockingQueue cursor timed out waiting for data")); + } resultBatch = queue.poll(thisTimeout, TimeUnit.MILLISECONDS); } else { resultBatch = queue.take(); } } - return true; + return resultBatch != null; } @Override diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index 97e58156ccb1..cdc1e2f4bd0a 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -23,24 +23,46 @@ import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.logger.Logger; +import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Iterator; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeoutException; import java.util.function.BinaryOperator; public class ParallelMergeCombiningSequenceTest { private static final Logger LOG = new Logger(ParallelMergeCombiningSequenceTest.class); + private static final int DEFAULT_TEST_YIELD_AFTER = 1024; + private static final int DEFAULT_TEST_BATCH_SIZE = 128; private ForkJoinPool pool; + private final Ordering ordering = Ordering.natural().onResultOf(p -> p.lhs); + private final BinaryOperator mergeFn = (lhs, rhs) -> { + if (lhs == null) { + return rhs; + } + + if (rhs == null) { + return lhs; + } + + return new IntPair(lhs.lhs, lhs.rhs + rhs.rhs); + }; + @Before public void setup() { @@ -53,318 +75,413 @@ public void teardown() pool.shutdown(); } + @Rule + public ExpectedException expectedException = ExpectedException.none(); + @Test - public void testNone() throws Exception + public void testOrderedResultBatchFromSequence() throws IOException { - List> input = new ArrayList<>(); - assertResult(input); + Sequence rawSequence = generateOrderedPairsSequence(5000); + Yielder> batchYielder = + ParallelMergeCombiningSequence.OrderedResultBatch.fromSequence(rawSequence, 128); + + ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = + new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>(batchYielder, ordering); + + Yielder rawYielder = Yielders.each(rawSequence); + + IntPair prev = null; + while (!rawYielder.isDone() && !cursor.isDone()) { + Assert.assertEquals(rawYielder.get(), cursor.get()); + LOG.debug("%s", cursor.get()); + Assert.assertNotEquals(cursor.get(), prev); + prev = cursor.get(); + rawYielder = rawYielder.next(rawYielder.get()); + cursor.advance(); + } + cursor.close(); + rawYielder.close(); } @Test - public void testEmptySerial() throws Exception + public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws IOException { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 1) + final int batchSize = 128; + final int sequenceSize = 5_000; + Sequence rawSequence = generateOrderedPairsSequence(sequenceSize); + Yielder> batchYielder = + ParallelMergeCombiningSequence.OrderedResultBatch.fromSequence(rawSequence, batchSize); + + ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = + new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>(batchYielder, ordering); + + + Yielder rawYielder = Yielders.each(rawSequence); + + ArrayBlockingQueue> outputQueue = + new ArrayBlockingQueue<>((int) Math.ceil(((double) sequenceSize / batchSize) + 2)); + + IntPair prev = null; + ParallelMergeCombiningSequence.OrderedResultBatch currentBatch = + new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + int batchCounter = 0; + while (!rawYielder.isDone() && !cursor.isDone()) { + Assert.assertEquals(rawYielder.get(), cursor.get()); + LOG.debug("%s", cursor.get()); + Assert.assertNotEquals(cursor.get(), prev); + prev = cursor.get(); + currentBatch.add(prev); + batchCounter++; + if (batchCounter >= batchSize) { + outputQueue.offer(currentBatch); + currentBatch = new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + batchCounter = 0; + } + rawYielder = rawYielder.next(rawYielder.get()); + cursor.advance(); + } + if (!currentBatch.isDrained()) { + outputQueue.offer(currentBatch); + } + outputQueue.offer(new ParallelMergeCombiningSequence.OrderedResultBatch<>()); + + rawYielder.close(); + cursor.close(); + + rawYielder = Yielders.each(rawSequence); + + Sequence queueAsSequence = ParallelMergeCombiningSequence.makeOutputSequenceForQueue( + outputQueue, + true, + System.currentTimeMillis() + 10_000, + new ParallelMergeCombiningSequence.CancellationGizmo() ); + + Yielder queueYielder = Yielders.each(queueAsSequence); + + int rowCtr = 0; + while (!rawYielder.isDone() && !queueYielder.isDone()) { + Assert.assertEquals(rawYielder.get(), queueYielder.get()); + LOG.debug("row %s: %s", rowCtr++, queueYielder.get()); + Assert.assertNotEquals(queueYielder.get(), prev); + prev = queueYielder.get(); + rawYielder = rawYielder.next(rawYielder.get()); + queueYielder = queueYielder.next(queueYielder.get()); + } + + rawYielder.close(); + queueYielder.close(); + } + + @Test + public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOException + { + final int batchSize = 128; + final int sequenceSize = 5_000; + Sequence rawSequence = generateOrderedPairsSequence(sequenceSize); + Yielder> batchYielder = + ParallelMergeCombiningSequence.OrderedResultBatch.fromSequence(rawSequence, batchSize); + + ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = + new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>(batchYielder, ordering); + + + Yielder rawYielder = Yielders.each(rawSequence); + + ArrayBlockingQueue> outputQueue = + new ArrayBlockingQueue<>((int) Math.ceil(((double) sequenceSize / batchSize) + 2)); + + IntPair prev = null; + ParallelMergeCombiningSequence.OrderedResultBatch currentBatch = + new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + int batchCounter = 0; + while (!rawYielder.isDone() && !cursor.isDone()) { + Assert.assertEquals(rawYielder.get(), cursor.get()); + LOG.debug("%s", cursor.get()); + Assert.assertNotEquals(cursor.get(), prev); + prev = cursor.get(); + currentBatch.add(prev); + batchCounter++; + if (batchCounter >= batchSize) { + outputQueue.offer(currentBatch); + currentBatch = new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + batchCounter = 0; + } + rawYielder = rawYielder.next(rawYielder.get()); + cursor.advance(); + } + if (!currentBatch.isDrained()) { + outputQueue.offer(currentBatch); + } + outputQueue.offer(new ParallelMergeCombiningSequence.OrderedResultBatch<>()); + + rawYielder.close(); + cursor.close(); + + rawYielder = Yielders.each(rawSequence); + + ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor queueCursor = + new ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor<>( + outputQueue, + ordering, + false, + -1L + ); + queueCursor.initialize(); + prev = null; + while (!rawYielder.isDone() && !queueCursor.isDone()) { + Assert.assertEquals(rawYielder.get(), queueCursor.get()); + LOG.debug("%s", queueCursor.get()); + Assert.assertNotEquals(queueCursor.get(), prev); + prev = queueCursor.get(); + rawYielder = rawYielder.next(rawYielder.get()); + queueCursor.advance(); + } + rawYielder.close(); + queueCursor.close(); + } + + @Test + public void testNone() throws Exception + { List> input = new ArrayList<>(); - input.add(Sequences.empty()); - input.add(Sequences.simple(pairs1)); assertResult(input); } @Test - public void testEmpty() throws Exception + public void testEmpties() throws Exception { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 1) - ); + // below min threshold, so will merge serially List> input = new ArrayList<>(); input.add(Sequences.empty()); input.add(Sequences.empty()); + assertResult(input); + + // above min sequence count threshold, so will merge in parallel (if enough cores) input.add(Sequences.empty()); input.add(Sequences.empty()); input.add(Sequences.empty()); - input.add(Sequences.simple(pairs1)); assertResult(input); } @Test - public void testSimpleSerial() throws Exception + public void testEmptiesAndNonEmpty() throws Exception { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 1) - ); + // below min threshold, so will merge serially + List> input = new ArrayList<>(); + input.add(Sequences.empty()); + input.add(generateOrderedPairsSequence(5)); + assertResult(input); - List pairs2 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 2) - ); + input.clear(); + // above min sequence count threshold, so will merge in parallel (if enough cores) + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(Sequences.empty()); + input.add(generateOrderedPairsSequence(5)); + assertResult(input); + } + @Test + public void testAllInSingleBatch() throws Exception + { + // below min threshold, so will merge serially List> input = new ArrayList<>(); - input.add(Sequences.simple(pairs1)); - input.add(Sequences.simple(pairs2)); - - assertResult(input); + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(6)); + assertResult(input, 10, 20); + + input.clear(); + + // above min sequence count threshold, so will merge in parallel (if enough cores) + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(6)); + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(8)); + input.add(generateOrderedPairsSequence(4)); + input.add(generateOrderedPairsSequence(6)); + assertResult(input, 10, 20); } @Test - public void testSimpleParallel() throws Exception + public void testAllInSingleYield() throws Exception { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 1) - ); + // below min threshold, so will merge serially + List> input = new ArrayList<>(); + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(6)); + assertResult(input, 4, 20); + + input.clear(); + + // above min sequence count threshold, so will merge in parallel (if enough cores) + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(6)); + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(8)); + input.add(generateOrderedPairsSequence(4)); + input.add(generateOrderedPairsSequence(6)); + assertResult(input, 4, 20); + } - List pairs2 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 2) - ); - List pairs3 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 2) - ); + @Test + public void testMultiBatchMultiYield() throws Exception + { + // below min threshold, so will merge serially + List> input = new ArrayList<>(); + input.add(generateOrderedPairsSequence(15)); + input.add(generateOrderedPairsSequence(26)); - List pairs4 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 2) - ); + assertResult(input, 5, 10); + + // above min sequence count threshold, so will merge in parallel (if enough cores) + input.add(generateOrderedPairsSequence(15)); + input.add(generateOrderedPairsSequence(33)); + input.add(generateOrderedPairsSequence(17)); + input.add(generateOrderedPairsSequence(14)); + assertResult(input, 5, 10); + } + @Test + public void testMixedSingleAndMultiYield() throws Exception + { + // below min threshold, so will merge serially List> input = new ArrayList<>(); - input.add(Sequences.simple(pairs1)); - input.add(Sequences.simple(pairs2)); - input.add(Sequences.simple(pairs3)); - input.add(Sequences.simple(pairs4)); + input.add(generateOrderedPairsSequence(60)); + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(8)); - assertResult(input); + assertResult(input, 5, 10); + + // above min sequence count threshold, so will merge in parallel (if enough cores) + input.add(generateOrderedPairsSequence(1)); + input.add(generateOrderedPairsSequence(8)); + input.add(generateOrderedPairsSequence(32)); + + assertResult(input, 5, 10); } @Test - public void testLongBoySerial() throws Exception + public void testLongerSequencesJustForFun() throws Exception { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 11), - new IntPair(7, 11), - new IntPair(9, 11), - new IntPair(11, 11), - new IntPair(16, 11), - new IntPair(24, 11), - new IntPair(25, 11), - new IntPair(27, 11), - new IntPair(28, 1) - ); - - List pairs2 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 1), - new IntPair(7, 1), - new IntPair(9, 1), - new IntPair(10, 1), - new IntPair(13, 1), - new IntPair(14, 1), - new IntPair(23, 1), - new IntPair(25, 1), - new IntPair(27, 1), - new IntPair(28, 2) - ); List> input = new ArrayList<>(); - input.add(Sequences.simple(pairs1)); - input.add(Sequences.simple(pairs2)); + input.add(generateOrderedPairsSequence(10_000)); + input.add(generateOrderedPairsSequence(9_001)); - assertResult(input); + assertResult(input, 128, 1024); + + input.add(generateOrderedPairsSequence(7_777)); + input.add(generateOrderedPairsSequence(8_500)); + input.add(generateOrderedPairsSequence(5_000)); + input.add(generateOrderedPairsSequence(8_888)); + + assertResult(input, 128, 1024); } @Test - public void testLongBoyParallel() throws Exception + public void testExceptionOnInputSequenceRead() throws Exception { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 11), - new IntPair(7, 11), - new IntPair(9, 11), - new IntPair(11, 11), - new IntPair(16, 11), - new IntPair(24, 11), - new IntPair(25, 11), - new IntPair(27, 11), - new IntPair(28, 1) - ); - - List pairs2 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 1), - new IntPair(7, 1), - new IntPair(9, 1), - new IntPair(10, 1), - new IntPair(13, 1), - new IntPair(14, 1), - new IntPair(23, 1), - new IntPair(25, 1), - new IntPair(27, 1), - new IntPair(28, 2) - ); + List> input = new ArrayList<>(); - List pairs3 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 1), - new IntPair(7, 6), - new IntPair(9, 1), - new IntPair(10, 2), - new IntPair(13, 1), - new IntPair(14, 3), - new IntPair(15, 1), - new IntPair(19, 7), - new IntPair(21, 1), - new IntPair(22, 2) - ); + input.add(explodingSequence(15)); + input.add(generateOrderedPairsSequence(25)); - List pairs4 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 1), - new IntPair(7, 4), - new IntPair(9, 1), - new IntPair(10, 2), - new IntPair(13, 1), - new IntPair(14, 11), - new IntPair(16, 1), - new IntPair(17, 13), - new IntPair(20, 1), - new IntPair(22, 2) - ); - List pairs5 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(5, 1), - new IntPair(7, 1), - new IntPair(9, 1), - new IntPair(10, 1), - new IntPair(13, 1), - new IntPair(14, 1), - new IntPair(16, 1), - new IntPair(25, 7), - new IntPair(27, 1), - new IntPair(30, 2) + expectedException.expect(RuntimeException.class); + expectedException.expectMessage( + "exploded" ); + assertException(input); + input.add(generateOrderedPairsSequence(5)); + input.add(generateOrderedPairsSequence(25)); + input.add(explodingSequence(11)); + input.add(generateOrderedPairsSequence(12)); - List> input = new ArrayList<>(); - input.add(Sequences.simple(pairs1)); - input.add(Sequences.simple(pairs2)); - input.add(Sequences.simple(pairs3)); - input.add(Sequences.simple(pairs4)); - input.add(Sequences.simple(pairs5)); - - assertResult(input); + assertException(input); } @Test - public void testMixedLength() throws Exception + public void testExceptionFirstResultFromSequence() throws Exception { - List pairs1 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 1) + List> input = new ArrayList<>(); + input.add(explodingSequence(0)); + input.add(generateOrderedPairsSequence(2)); + input.add(generateOrderedPairsSequence(2)); + input.add(generateOrderedPairsSequence(2)); + + expectedException.expect(RuntimeException.class); + expectedException.expectMessage( + "exploded" ); + assertException(input); + } - List pairs2 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(6, 2), - new IntPair(10, 2) + @Test + public void testExceptionFirstResultFromMultipleSequence() throws Exception + { + List> input = new ArrayList<>(); + input.add(explodingSequence(0)); + input.add(explodingSequence(0)); + input.add(explodingSequence(0)); + input.add(generateOrderedPairsSequence(2)); + input.add(generateOrderedPairsSequence(2)); + input.add(generateOrderedPairsSequence(2)); + + expectedException.expect(RuntimeException.class); + expectedException.expectMessage( + "exploded" ); + assertException(input); + } - List pairs3 = Arrays.asList( - new IntPair(4, 5), - new IntPair(10, 3) - ); + @Test + public void testTimeoutExceptionDueToStalledInput() throws Exception + { + final int someSize = 2048; + List> input = new ArrayList<>(); + input.add(generateOrderedPairsSequence(someSize)); + input.add(generateOrderedPairsSequence(someSize)); + input.add(generateOrderedPairsSequence(someSize)); + input.add(slowSequence(someSize, 500)); - List pairs4 = Arrays.asList( - new IntPair(0, 1), - new IntPair(1, 13), - new IntPair(4, 1), - new IntPair(6, 2), - new IntPair(10, 2) - ); + expectedException.expect(RuntimeException.class); + expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); + expectedException.expectMessage("Sequence iterator timed out waiting for data"); - List pairs5 = Arrays.asList( - new IntPair(0, 6), - new IntPair(1, 1), - new IntPair(2, 1), - new IntPair(5, 11), - new IntPair(6, 1) - ); + assertException(input, DEFAULT_TEST_BATCH_SIZE, DEFAULT_TEST_YIELD_AFTER, 1000L, 0); + } + @Test + public void testTimeoutExceptionDueToStalledReader() throws Exception + { + final int someSize = 2048; List> input = new ArrayList<>(); - input.add(Sequences.simple(pairs1)); - input.add(Sequences.simple(pairs2)); - input.add(Sequences.simple(pairs3)); - input.add(Sequences.simple(pairs4)); - input.add(Sequences.simple(pairs5)); - - assertResult(input); + input.add(generateOrderedPairsSequence(someSize)); + input.add(generateOrderedPairsSequence(someSize)); + input.add(generateOrderedPairsSequence(someSize)); + input.add(generateOrderedPairsSequence(someSize)); + + expectedException.expect(RuntimeException.class); + expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); + expectedException.expectMessage("Sequence iterator timed out"); + assertException(input, 8, 64, 1000, 500); } - private void assertResult(List> sequences) throws InterruptedException + private void assertResult(List> sequences) throws InterruptedException, IOException { - final Ordering ordering = Ordering.natural().onResultOf(p -> p.lhs); - final BinaryOperator mergeFn = (lhs, rhs) -> { - if (lhs == null) { - return rhs; - } - - if (rhs == null) { - return lhs; - } - - return new IntPair(lhs.lhs, lhs.rhs + rhs.rhs); - }; + assertResult(sequences, DEFAULT_TEST_BATCH_SIZE, DEFAULT_TEST_YIELD_AFTER); + } + private void assertResult(List> sequences, int batchSize, int yieldAfter) + throws InterruptedException, IOException + { final CombiningSequence combiningSequence = CombiningSequence.create( new MergeSequence<>(ordering, Sequences.simple(sequences)), ordering, @@ -376,13 +493,12 @@ private void assertResult(List> sequences) throws InterruptedE sequences, ordering, mergeFn, - 64, - false, + true, 5000, 0, Runtime.getRuntime().availableProcessors() - 1, - 4, - 2 + yieldAfter, + batchSize ); Yielder combiningYielder = Yielders.each(combiningSequence); @@ -392,7 +508,7 @@ private void assertResult(List> sequences) throws InterruptedE while (!combiningYielder.isDone() && !parallelMergeCombineYielder.isDone()) { Assert.assertEquals(combiningYielder.get(), parallelMergeCombineYielder.get()); - LOG.info("%s", parallelMergeCombineYielder.get()); + LOG.debug("%s", parallelMergeCombineYielder.get()); Assert.assertNotEquals(parallelMergeCombineYielder.get(), prev); prev = parallelMergeCombineYielder.get(); combiningYielder = combiningYielder.next(combiningYielder.get()); @@ -405,6 +521,57 @@ private void assertResult(List> sequences) throws InterruptedE Thread.sleep(100); } Assert.assertEquals(0, pool.getRunningThreadCount()); + combiningYielder.close(); + parallelMergeCombineYielder.close(); + } + + private void assertException(List> sequences) throws Exception + { + assertException(sequences, DEFAULT_TEST_BATCH_SIZE, DEFAULT_TEST_YIELD_AFTER, 5000L, 0); + } + + private void assertException( + List> sequences, + int batchSize, + int yieldAfter, + long timeout, + int readDelayMillis + ) + throws Exception + { + try { + final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( + pool, + sequences, + ordering, + mergeFn, + true, + timeout, + 0, + Runtime.getRuntime().availableProcessors() - 1, + yieldAfter, + batchSize + ); + + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + + IntPair prev = null; + + while (!parallelMergeCombineYielder.isDone()) { + LOG.debug("%s", parallelMergeCombineYielder.get()); + Assert.assertNotEquals(parallelMergeCombineYielder.get(), prev); + prev = parallelMergeCombineYielder.get(); + if (readDelayMillis > 0 && ThreadLocalRandom.current().nextBoolean()) { + Thread.sleep(readDelayMillis); + } + parallelMergeCombineYielder = parallelMergeCombineYielder.next(parallelMergeCombineYielder.get()); + } + parallelMergeCombineYielder.close(); + } + catch (Exception ex) { + LOG.warn(ex, "actual exception:"); + throw ex; + } } static class IntPair extends Pair @@ -414,4 +581,105 @@ static class IntPair extends Pair super(lhs, rhs); } } + + private static List generateOrderedPairs(int length) + { + int counter = 0; + int i = 0; + List generatedSequence = new ArrayList<>(length); + while (counter < length) { + i++; + if (ThreadLocalRandom.current().nextBoolean()) { + generatedSequence.add(new IntPair(i, ThreadLocalRandom.current().nextInt(1, 100))); + counter++; + } + } + return generatedSequence; + } + + private static Sequence generateOrderedPairsSequence(int length) + { + return Sequences.simple(generateOrderedPairs(length)); + } + + private static Sequence explodingSequence(int explodeAfter) + { + List items = generateOrderedPairs(explodeAfter + 1); + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + int i = 0; + @Override + public boolean hasNext() + { + return i < items.size(); + } + + @Override + public IntPair next() + { + if (i == explodeAfter) { + throw new RuntimeException("exploded"); + } + return items.get(i++); + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } + } + ); + } + + private static Sequence slowSequence(int size, int delay) + { + List items = generateOrderedPairs(size); + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + int i = 0; + @Override + public boolean hasNext() + { + return i < items.size(); + } + + @Override + public IntPair next() + { + if (ThreadLocalRandom.current().nextBoolean()) { + try { + Thread.sleep(delay); + } + catch (InterruptedException ex) { + throw new RuntimeException(ex); + } + } + return items.get(i++); + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } + } + ); + } } diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java index 6ec31083ba40..ca02de7156df 100644 --- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java +++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java @@ -32,6 +32,9 @@ import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.query.context.ResponseContext; +import java.util.Comparator; +import java.util.function.BinaryOperator; + public class MaterializedViewQueryQueryToolChest extends QueryToolChest { private final QueryToolChestWarehouse warehouse; @@ -58,6 +61,20 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) }; } + @Override + public BinaryOperator createMergeFn(Query query) + { + final Query realQuery = getRealQuery(query); + return warehouse.getToolChest(realQuery).createMergeFn(realQuery); + } + + @Override + public Comparator createResultComparator(Query query) + { + final Query realQuery = getRealQuery(query); + return warehouse.getToolChest(realQuery).createResultComparator(realQuery); + } + @Override public QueryMetrics makeMetrics(Query query) { diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 05d3b2352382..f9bb0f273158 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -51,6 +51,7 @@ import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DataSource; +import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -351,6 +352,14 @@ public long getMaxQueuedBytes() return 0L; } }, + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + }, ForkJoinPool.commonPool() ); diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index cc48de883e07..8d3cc07d7afd 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -146,6 +146,12 @@ public String getTmpDir() return System.getProperty("java.io.tmpdir"); } + @Config(value = "${base_path}.useParallelMergePool") + public boolean useParallelMergePool() + { + return true; + } + @Config(value = "${base_path}.numMergePoolThreads") public int getNumThreadsMergePoolConfigured() { diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 0a2d362ee016..88e29cec3614 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -56,10 +56,10 @@ public class QueryContexts public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0; public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); public static final long NO_TIMEOUT = 0; - public static final boolean DEFAULT_ENABLE_PARALLEL_MERGE = false; + public static final boolean DEFAULT_ENABLE_PARALLEL_MERGE = true; public static final int DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS = 1024; public static final int DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS = 128; - public static final int DEFAULT_PARALLEL_MERGE_PARALLELISM = 3; + public static final int DEFAULT_PARALLEL_MERGE_PARALLELISM = Integer.MAX_VALUE; @SuppressWarnings("unused") // Used by Jackson serialization public enum Vectorize diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index 4c760b02cdff..511af9f4b3b6 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -113,9 +113,10 @@ public QueryRunner mergeResults(QueryRunner runner) * function is used in the default {@link ResultMergeQueryRunner} provided by * {@link QueryToolChest#mergeResults(QueryRunner)} and can be used in additional future merge implementations */ + @Nullable public BinaryOperator createMergeFn(Query query) { - throw new UOE("%s doesn't provide a merge function", query.getClass().getName()); + return null; } /** diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java index 6125dd0897b4..dca39eb56e20 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java @@ -113,7 +113,7 @@ Sequence mergeResults( @Nullable default BinaryOperator createMergeFn(Query query) { - throw new UOE("%s doesn't provide a merge function", this.getClass().getName()); + return null; } /** diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 989abc2dec6f..1b9324530396 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -62,6 +62,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -130,21 +131,34 @@ public Sequence doRun( private Ordering makeOrdering(SegmentMetadataQuery query) { - if (query.isMerge()) { - // Merge everything always - return Comparators.alwaysEqual(); - } - - return query.getResultOrdering(); // No two elements should be equal, so it should never merge + return (Ordering) SegmentMetadataQueryQueryToolChest.this.createResultComparator(query); } private BinaryOperator createMergeFn(final SegmentMetadataQuery inQ) { - return (arg1, arg2) -> mergeAnalyses(arg1, arg2, inQ.isLenientAggregatorMerge()); + return SegmentMetadataQueryQueryToolChest.this.createMergeFn(inQ); } }; } + @Override + public BinaryOperator createMergeFn(Query query) + { + return (arg1, arg2) -> mergeAnalyses(arg1, arg2, ((SegmentMetadataQuery) query).isLenientAggregatorMerge()); + } + + @Override + public Comparator createResultComparator(Query query) + { + SegmentMetadataQuery segmentMetadataQuery = (SegmentMetadataQuery) query; + if (segmentMetadataQuery.isMerge()) { + // Merge everything always + return Comparators.alwaysEqual(); + } + + return segmentMetadataQuery.getResultOrdering(); // No two elements should be equal, so it should never merge + } + @Override public QueryMetrics> makeMetrics(SegmentMetadataQuery query) { diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 2c814cf599bb..9aac1114adca 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -52,6 +52,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.BySegmentResultValueClass; import org.apache.druid.query.CacheStrategy; +import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -90,6 +91,7 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.ForkJoinPool; +import java.util.function.BinaryOperator; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -105,6 +107,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final CachePopulator cachePopulator; private final CacheConfig cacheConfig; private final DruidHttpClientConfig httpClientConfig; + private final DruidProcessingConfig processingConfig; private final ForkJoinPool pool; @Inject @@ -116,6 +119,7 @@ public CachingClusteredClient( CachePopulator cachePopulator, CacheConfig cacheConfig, @Client DruidHttpClientConfig httpClientConfig, + DruidProcessingConfig processingConfig, @Merging ForkJoinPool pool ) { @@ -126,6 +130,7 @@ public CachingClusteredClient( this.cachePopulator = cachePopulator; this.cacheConfig = cacheConfig; this.httpClientConfig = httpClientConfig; + this.processingConfig = processingConfig; this.pool = pool; if (cacheConfig.isQueryCacheable(Query.GROUP_BY) && (cacheConfig.isUseCache() || cacheConfig.isPopulateCache())) { @@ -298,15 +303,15 @@ Sequence run(final UnaryOperator> time private Sequence merge(List> sequencesByInterval) { - if (QueryContexts.getEnableParallelMerges(query)) { + BinaryOperator mergeFn = toolChest.createMergeFn(query); + if (processingConfig.useParallelMergePool() && QueryContexts.getEnableParallelMerges(query) && mergeFn != null) { final int yieldAfter = QueryContexts.getParallelMergeInitialYieldRows(query); final int batchSize = QueryContexts.getParallelMergeSmallBatchRows(query); return new ParallelMergeCombiningSequence<>( pool, sequencesByInterval, query.getResultOrdering(), - toolChest.createMergeFn(query), - 8 * (yieldAfter / batchSize), + mergeFn, QueryContexts.hasTimeout(query), QueryContexts.getTimeout(query), QueryContexts.getPriority(query), diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index a915bed0443a..d36471ecef78 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.DataSource; +import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -308,13 +309,22 @@ public int getCacheBulkMergeLimit() return mergeLimit; } }, - new DruidHttpClientConfig() { + new DruidHttpClientConfig() + { @Override public long getMaxQueuedBytes() { return 0L; } }, + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + }, ForkJoinPool.commonPool() ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index d8b0d2774e23..6c74315caa11 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -72,6 +72,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.BySegmentResultValueClass; import org.apache.druid.query.DataSource; +import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Druids; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.Query; @@ -2758,6 +2759,14 @@ public long getMaxQueuedBytes() return 0L; } }, + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + }, ForkJoinPool.commonPool() ); } From 6c2e1157b4959e436b1ae1651c42af1ad2263e90 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 1 Oct 2019 12:14:07 -0700 Subject: [PATCH 15/51] remove unused var --- .../common/guava/ParallelMergeCombiningSequence.java | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index c08e04c6708c..34bc2b66c187 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -422,7 +422,6 @@ protected void compute() int counter = 0; int batchCounter = 0; OrderedResultBatch outputBatch = new OrderedResultBatch<>(batchSize); - OrderedResultBatch errorBatch = null; T currentCombinedValue = initialValue; while (counter++ < yieldAfter && !pQueue.isEmpty()) { @@ -492,12 +491,8 @@ protected void compute() depth + 1, cancellationGizmo )); - } else if (cancellationGizmo.isCancelled() || errorBatch != null) { - if (errorBatch == null) { - outputQueue.offer(new OrderedResultBatch<>()); - } else { - outputQueue.offer(errorBatch); - } + } else if (cancellationGizmo.isCancelled()) { + outputQueue.offer(new OrderedResultBatch<>()); } else { // if priority queue is empty, push the final accumulated value outputBatch.add(currentCombinedValue); From e3cecdc1ddc555e2683b95cead30a94d08536527 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 1 Oct 2019 18:18:45 -0700 Subject: [PATCH 16/51] comments --- .../common/guava/ParallelMergeCombiningSequence.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 34bc2b66c187..609b6aee2623 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -473,6 +473,10 @@ protected void compute() if (!outputBatch.isDrained()) { outputQueue.offer(outputBatch); } + + // measure the time it took to process 'yieldAfter' elements in order to project a next 'yieldAfter' value + // which we want to target a 10ms task run time. smooth this value with a cumulative moving average in order + // to prevent normal jitter in processing time from skewing the next yield value too far in any direction final long elapsedMillis = Math.max( TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS), 1L @@ -480,6 +484,7 @@ protected void compute() final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); final double cumulativeMovingAverage = (nextYieldAfter + (depth * yieldAfter)) / (depth + 1); final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); + getPool().execute(new MergeCombineAction<>( pQueue, outputQueue, @@ -492,9 +497,11 @@ protected void compute() cancellationGizmo )); } else if (cancellationGizmo.isCancelled()) { + // if we got the cancellation signal, go ahead and write terminal value into output queue to help gracefully + // allow downstream stuff to stop outputQueue.offer(new OrderedResultBatch<>()); } else { - // if priority queue is empty, push the final accumulated value + // if priority queue is empty, push the final accumulated value into the output batch and push it out outputBatch.add(currentCombinedValue); outputQueue.offer(outputBatch); // ... and the terminal value to indicate the blocking queue holding the values is complete From b01ddbac6e4dc55883ac4b3b603e51bdd6a9c7b1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 2 Oct 2019 01:36:59 -0700 Subject: [PATCH 17/51] timeboundary mergeFn --- .../TimeBoundaryQueryQueryToolChest.java | 25 ++++++++++++++++++ .../timeboundary/TimeBoundaryResultValue.java | 7 ++--- .../client/CachingClusteredClientTest.java | 26 +++++++------------ 3 files changed, 37 insertions(+), 21 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 6499b88b99cd..d6a402fcc83b 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -23,6 +23,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.inject.Inject; import org.apache.druid.java.util.common.DateTimes; @@ -43,7 +45,9 @@ import org.apache.druid.timeline.LogicalSegment; import java.nio.ByteBuffer; +import java.util.Comparator; import java.util.List; +import java.util.function.BinaryOperator; import java.util.stream.Collectors; /** @@ -112,6 +116,27 @@ protected Sequence> doRun( }; } + @Override + public BinaryOperator> createMergeFn(Query> query) + { + TimeBoundaryQuery boundQuery = (TimeBoundaryQuery) query; + return (result1, result2) -> { + final List> mergeList; + if (result1 == null) { + mergeList = result2 != null ? ImmutableList.of(result2) : null; + } else { + mergeList = result2 != null ? ImmutableList.of(result1, result2) : ImmutableList.of(result1); + } + return Iterables.getOnlyElement(boundQuery.mergeResults(mergeList)); + }; + } + + @Override + public Comparator> createResultComparator(Query> query) + { + return query.getResultOrdering(); + } + @Override public QueryMetrics> makeMetrics(TimeBoundaryQuery query) { diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryResultValue.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryResultValue.java index d5a1c447ad5b..ae7240c4156e 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryResultValue.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryResultValue.java @@ -27,6 +27,7 @@ import javax.annotation.Nullable; import java.util.Map; +import java.util.Objects; /** */ @@ -80,11 +81,7 @@ public boolean equals(Object o) TimeBoundaryResultValue that = (TimeBoundaryResultValue) o; - if (value != null ? !value.equals(that.value) : that.value != null) { - return false; - } - - return true; + return Objects.equals(value, that.value); } @Override diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 6c74315caa11..da49f53703d0 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -1572,16 +1572,13 @@ public void testTimeBoundaryCaching() .bound(TimeBoundaryQuery.MAX_TIME) .build(), Intervals.of("2011-01-01/2011-01-02"), - makeTimeBoundaryResult(DateTimes.of("2011-01-01"), null, DateTimes.of("2011-01-02")), + makeTimeBoundaryResult(DateTimes.of("2011-01-02"), null, DateTimes.of("2011-01-02")), Intervals.of("2011-01-01/2011-01-03"), - makeTimeBoundaryResult(DateTimes.of("2011-01-02"), null, DateTimes.of("2011-01-03")), + makeTimeBoundaryResult(DateTimes.of("2011-01-03"), null, DateTimes.of("2011-01-03")), Intervals.of("2011-01-01/2011-01-10"), - makeTimeBoundaryResult(DateTimes.of("2011-01-05"), null, DateTimes.of("2011-01-10")), - - Intervals.of("2011-01-01/2011-01-10"), - makeTimeBoundaryResult(DateTimes.of("2011-01-05T01"), null, DateTimes.of("2011-01-10")) + makeTimeBoundaryResult(DateTimes.of("2011-01-10"), null, DateTimes.of("2011-01-10")) ); testQueryCaching( @@ -1785,19 +1782,19 @@ private Iterable> makeTimeBoundaryResult( if (minTime != null && maxTime != null) { value = ImmutableMap.of( TimeBoundaryQuery.MIN_TIME, - minTime.toString(), + minTime, TimeBoundaryQuery.MAX_TIME, - maxTime.toString() + maxTime ); } else if (maxTime != null) { value = ImmutableMap.of( TimeBoundaryQuery.MAX_TIME, - maxTime.toString() + maxTime ); } else { value = ImmutableMap.of( TimeBoundaryQuery.MIN_TIME, - minTime.toString() + minTime ); } @@ -3040,16 +3037,13 @@ public void testTimeBoundaryCachingWhenTimeIsInteger() .bound(TimeBoundaryQuery.MAX_TIME) .build(), Intervals.of("1970-01-01/2011-01-02"), - makeTimeBoundaryResult(DateTimes.of("1970-01-01"), null, DateTimes.of("1970-01-02")), + makeTimeBoundaryResult(DateTimes.of("1970-01-02"), null, DateTimes.of("1970-01-02")), Intervals.of("1970-01-01/2011-01-03"), - makeTimeBoundaryResult(DateTimes.of("1970-01-02"), null, DateTimes.of("1970-01-03")), - - Intervals.of("1970-01-01/2011-01-10"), - makeTimeBoundaryResult(DateTimes.of("1970-01-05"), null, DateTimes.of("1970-01-10")), + makeTimeBoundaryResult(DateTimes.of("1970-01-03"), null, DateTimes.of("1970-01-03")), Intervals.of("1970-01-01/2011-01-10"), - makeTimeBoundaryResult(DateTimes.of("1970-01-05T01"), null, DateTimes.of("1970-01-10")) + makeTimeBoundaryResult(DateTimes.of("1970-01-10"), null, DateTimes.of("1970-01-10")) ); testQueryCaching( From 78f1da849293aae5c0ae6335f15f52fc0762ae6c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 2 Oct 2019 04:19:03 -0700 Subject: [PATCH 18/51] simplify, more javadoc --- .../guava/ParallelMergeCombiningSequence.java | 432 ++++++++---------- .../ParallelMergeCombiningSequenceTest.java | 49 +- 2 files changed, 218 insertions(+), 263 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 609b6aee2623..fb649a0c7a05 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -98,8 +98,8 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat return Sequences.empty().toYielder(initValue, accumulator); } - final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); - ParallelMergeCombineAction finalMergeAction = new ParallelMergeCombineAction<>( + final BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); + MergeCombinePartitioningAction finalMergeAction = new MergeCombinePartitioningAction<>( baseSequences, orderingFn, combineFn, @@ -118,16 +118,16 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat } /** - * Create an output {@link Sequence} that wraps the output {@link BlockingQueue} of a {@link RecursiveAction} task + * Create an output {@link Sequence} that wraps the output {@link BlockingQueue} of a + * {@link MergeCombinePartitioningAction} */ static Sequence makeOutputSequenceForQueue( - BlockingQueue> queue, + BlockingQueue> queue, boolean hasTimeout, long timeoutAt, CancellationGizmo cancellationGizmo ) { - return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -136,7 +136,7 @@ public Iterator make() { return new Iterator() { - private OrderedResultBatch currentBatch; + private ResultBatch currentBatch; @Override public boolean hasNext() @@ -208,18 +208,30 @@ public void cleanup(Iterator iterFromMake) } /** - * This {@link RecursiveAction} is the initial task of the parallel merge-combine process, it will partition the - * batched result yielders to do 2 layer parallel merge, spawning some number of {@link MergeCombineAction} directly - * for the yielders for the first layer, and spawning a single {@link BlockingQueueMergeCombineSeedAction} to wait for results - * to be available in the 'output' {@link BlockingQueue} of the first layer to do a final merge combine of all the - * parallel computed results. + * 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 ParallelMergeCombineAction extends RecursiveAction + private static class MergeCombinePartitioningAction extends RecursiveAction { private final List> sequences; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final BlockingQueue> out; + private final BlockingQueue> out; private final int queueSize; private final int parallelism; private final int yieldAfter; @@ -228,11 +240,11 @@ private static class ParallelMergeCombineAction extends RecursiveAction private final long timeoutAt; private final CancellationGizmo cancellationGizmo; - private ParallelMergeCombineAction( + private MergeCombinePartitioningAction( List> sequences, Ordering orderingFn, BinaryOperator combineFn, - BlockingQueue> out, + BlockingQueue> out, int queueSize, int parallelism, int yieldAfter, @@ -259,21 +271,26 @@ private ParallelMergeCombineAction( protected void compute() { try { - final int parallelMergeTasks = computeNumTasks(); + final int parallelTaskCount = computeNumTasks(); // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, // just serially perform the merge-combine with a single task - if (sequences.size() < 4 || parallelMergeTasks < 2) { + if (sequences.size() < 4 || parallelTaskCount < 2) { LOG.info( "Input sequence count (%s) or available parallel merge task count (%s) too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", sequences.size(), - parallelMergeTasks + parallelTaskCount ); - QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); - YielderMergeCombineSeedAction mergeAction = new YielderMergeCombineSeedAction<>( - sequences, + QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + + List> sequenceCursors = new ArrayList<>(sequences.size()); + for (Sequence s : sequences) { + sequenceCursors.add(new YielderBatchedResultsCursor<>(new SequenceBatcher<>(s, batchSize), orderingFn)); + } + PrepareMergeCombineInputsAction blockForInputsAction = new PrepareMergeCombineInputsAction<>( + sequenceCursors, resultsPusher, orderingFn, combineFn, @@ -281,23 +298,23 @@ protected void compute() batchSize, cancellationGizmo ); - getPool().execute(mergeAction); + getPool().execute(blockForInputsAction); } else { // 2 layer parallel merge done in fjp - LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelMergeTasks, sequences.size()); - spawnParallelTasks(parallelMergeTasks); + LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelTaskCount, sequences.size()); + spawnParallelTasks(parallelTaskCount); } } catch (Exception ex) { cancellationGizmo.cancel(ex); - out.offer(new OrderedResultBatch<>()); + out.offer(new ResultBatch<>()); } } void spawnParallelTasks(int parallelMergeTasks) { List tasks = new ArrayList<>(); - List>> intermediaryOutputs = new ArrayList<>(parallelMergeTasks); + List>> intermediaryOutputs = new ArrayList<>(parallelMergeTasks); List>> partitions = Lists.partition(sequences, sequences.size() / parallelMergeTasks); @@ -305,12 +322,16 @@ void spawnParallelTasks(int parallelMergeTasks) for (List> partition : partitions) { - BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); + BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); intermediaryOutputs.add(outputQueue); - QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); + QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); - YielderMergeCombineSeedAction mergeAction = new YielderMergeCombineSeedAction( - partition, + List> partitionCursors = new ArrayList<>(sequences.size()); + for (Sequence s : partition) { + partitionCursors.add(new YielderBatchedResultsCursor<>(new SequenceBatcher<>(s, batchSize), orderingFn)); + } + PrepareMergeCombineInputsAction blockForInputsAction = new PrepareMergeCombineInputsAction<>( + partitionCursors, pusher, orderingFn, combineFn, @@ -318,23 +339,27 @@ void spawnParallelTasks(int parallelMergeTasks) batchSize, cancellationGizmo ); - tasks.add(mergeAction); + tasks.add(blockForInputsAction); } for (RecursiveAction task : tasks) { getPool().execute(task); } - QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); - BlockingQueueMergeCombineSeedAction finalMergeAction = new BlockingQueueMergeCombineSeedAction<>( - intermediaryOutputs, + QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + List> intermediaryOutputsCursors = new ArrayList<>(intermediaryOutputs.size()); + for (BlockingQueue> queue : intermediaryOutputs) { + intermediaryOutputsCursors.add( + new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt) + ); + } + PrepareMergeCombineInputsAction finalMergeAction = new PrepareMergeCombineInputsAction<>( + intermediaryOutputsCursors, outputPusher, orderingFn, combineFn, yieldAfter, batchSize, - hasTimeout, - timeoutAt, cancellationGizmo ); @@ -374,7 +399,7 @@ int computeNumTasks() * * This task takes a 'yieldAfter' parameter which controls how many input result rows will be processed before this * task completes and executes a new task to continue where it left off. This value is initially set by the - * {@link ParallelMergeCombineAction} to a default value, but after that this process is timed to try and compute + * {@link MergeCombinePartitioningAction} to a default value, but after that this process is timed to try and compute * an 'optimal' number of rows to yield to achieve a task runtime of ~10ms, on the assumption that the time to process * n results will be approximately the same. */ @@ -383,7 +408,7 @@ private static class MergeCombineAction extends RecursiveAction private final PriorityQueue> pQueue; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; + private final QueuePusher> outputQueue; private final T initialValue; private final int yieldAfter; private final int batchSize; @@ -392,7 +417,7 @@ private static class MergeCombineAction extends RecursiveAction private MergeCombineAction( PriorityQueue> pQueue, - QueuePusher> outputQueue, + QueuePusher> outputQueue, Ordering orderingFn, BinaryOperator combineFn, T initialValue, @@ -421,7 +446,7 @@ protected void compute() int counter = 0; int batchCounter = 0; - OrderedResultBatch outputBatch = new OrderedResultBatch<>(batchSize); + ResultBatch outputBatch = new ResultBatch<>(batchSize); T currentCombinedValue = initialValue; while (counter++ < yieldAfter && !pQueue.isEmpty()) { @@ -456,7 +481,7 @@ protected void compute() batchCounter++; if (batchCounter >= batchSize) { outputQueue.offer(outputBatch); - outputBatch = new OrderedResultBatch<>(batchSize); + outputBatch = new ResultBatch<>(batchSize); batchCounter = 0; } @@ -499,133 +524,62 @@ protected void compute() } else if (cancellationGizmo.isCancelled()) { // if we got the cancellation signal, go ahead and write terminal value into output queue to help gracefully // allow downstream stuff to stop - outputQueue.offer(new OrderedResultBatch<>()); + outputQueue.offer(new ResultBatch<>()); } else { // if priority queue is empty, push the final accumulated value into the output batch and push it out outputBatch.add(currentCombinedValue); outputQueue.offer(outputBatch); // ... and the terminal value to indicate the blocking queue holding the values is complete - outputQueue.offer(new OrderedResultBatch<>()); + outputQueue.offer(new ResultBatch<>()); } } catch (Exception ex) { cancellationGizmo.cancel(ex); - outputQueue.offer(new OrderedResultBatch<>()); + outputQueue.offer(new ResultBatch<>()); } } } /** - * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link Yielder} - * in order to construct a set of {@link YielderBatchedResultsCursor} to feed to a {@link MergeCombineAction} - * which will do the actual work of merging and combining the result batches. - */ - private static class YielderMergeCombineSeedAction extends RecursiveAction - { - private final List> sequences; - private final Ordering orderingFn; - private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; - private final int yieldAfter; - private final int batchSize; - private final CancellationGizmo cancellationGizmo; - - private YielderMergeCombineSeedAction( - List> sequences, - QueuePusher> outputQueue, - Ordering orderingFn, - BinaryOperator combineFn, - int yieldAfter, - int batchSize, - CancellationGizmo cancellationGizmo - ) - { - this.sequences = sequences; - this.orderingFn = orderingFn; - this.combineFn = combineFn; - this.outputQueue = outputQueue; - this.yieldAfter = yieldAfter; - this.batchSize = batchSize; - this.cancellationGizmo = cancellationGizmo; - } - - @Override - protected void compute() - { - try { - PriorityQueue> cursors = new PriorityQueue<>(sequences.size()); - for (Sequence s : sequences) { - final SequenceYielder yielder = new SequenceYielder<>(s, batchSize); - final Yielder> batchYielder = yielder.getYielder(); - if (!(batchYielder.get() == null || (batchYielder.get().isDrained() && batchYielder.isDone()))) { - YielderBatchedResultsCursor batchedSequenceYielder = new YielderBatchedResultsCursor<>( - batchYielder, - orderingFn - ); - cursors.offer(batchedSequenceYielder); - } - } - - if (cursors.isEmpty()) { - outputQueue.offer(new OrderedResultBatch<>()); - } else { - getPool().execute(new MergeCombineAction( - cursors, - outputQueue, - orderingFn, - combineFn, - null, - yieldAfter, - batchSize, - 1, - cancellationGizmo - )); - } - } - catch (Exception ex) { - cancellationGizmo.cancel(ex); - outputQueue.offer(new OrderedResultBatch<>()); - } - } - } - - /** - * This {@link RecursiveAction} waits for {@link OrderedResultBatch} to be available in a set of {@link BlockingQueue} - * in order to construct a set of {@link BlockingQueueuBatchedResultsCursor} to feed to a {@link MergeCombineAction} - * which will do the actual work of merging and combining the result batches. + * This {@link RecursiveAction}, given a set of uninitialized {@link BatchedResultsCursor}, will initialize each of + * them (which is a potentially managed blocking operation) so that each will produce a {@link ResultBatch} + * from the {@link Yielder} or {@link BlockingQueue} that backs the cursor. + * + * Once initialized with a {@link ResultBatch}, the cursors are inserted into a {@link PriorityQueue} and + * fed into a {@link MergeCombineAction} which will do the actual work of merging and combining the result batches. + * This happens as soon as all cursors are initialized, as long as there is at least 1 cursor that is not 'done' + * ({@link BatchedResultsCursor#isDone()}). + * + * This task may take longer than other tasks on the {@link ForkJoinPool}, but is doing little actual work, the + * majority of its time will be spent managed blocking until results are ready for each cursor, or will be incredibly + * short lived if all inputs are already available. */ - private static class BlockingQueueMergeCombineSeedAction extends RecursiveAction + private static class PrepareMergeCombineInputsAction extends RecursiveAction { - private final List>> queues; + private final List> partition; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; + private final QueuePusher> outputQueue; private final int yieldAfter; private final int batchSize; - private final boolean hasTimeout; - private final long timeoutAt; private final CancellationGizmo cancellationGizmo; - private BlockingQueueMergeCombineSeedAction( - List>> queues, - QueuePusher> outputQueue, + private PrepareMergeCombineInputsAction( + List> partition, + QueuePusher> outputQueue, Ordering orderingFn, BinaryOperator combineFn, int yieldAfter, int batchSize, - boolean hasTimeout, - long timeoutAt, CancellationGizmo cancellationGizmo ) { - this.queues = queues; + this.partition = partition; this.orderingFn = orderingFn; this.combineFn = combineFn; this.outputQueue = outputQueue; this.yieldAfter = yieldAfter; this.batchSize = batchSize; - this.hasTimeout = hasTimeout; - this.timeoutAt = timeoutAt; this.cancellationGizmo = cancellationGizmo; } @@ -633,15 +587,12 @@ private BlockingQueueMergeCombineSeedAction( protected void compute() { try { - PriorityQueue> cursors = new PriorityQueue<>(queues.size()); - for (BlockingQueue> queue : queues) { - BatchedResultsCursor outputCursor = - new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt); - + PriorityQueue> cursors = new PriorityQueue<>(partition.size()); + for (BatchedResultsCursor cursor : partition) { // this is blocking - outputCursor.initialize(); - if (!outputCursor.isDone()) { - cursors.offer(outputCursor); + cursor.initialize(); + if (!cursor.isDone()) { + cursors.offer(cursor); } } @@ -658,87 +609,14 @@ protected void compute() cancellationGizmo )); } else { - outputQueue.offer(new OrderedResultBatch<>()); + outputQueue.offer(new ResultBatch<>()); } } catch (Exception ex) { cancellationGizmo.cancel(ex); - outputQueue.offer(new OrderedResultBatch<>()); - } - } - } - - - /** - * Token to allow any {@link RecursiveAction} signal the others and the output sequence that something bad happened - * and processing should cancel, such as a timeout or connection loss. - */ - static class CancellationGizmo - { - // volatile instead of AtomicBoolean because it is never unset - private volatile boolean cancelled; - private volatile Exception exception; - - void cancel(Exception ex) - { - if (cancelled) { - return; - } - cancelled = true; - exception = ex; - } - - boolean isCancelled() - { - return cancelled; - } - - RuntimeException getRuntimeException() - { - if (exception instanceof RuntimeException) { - return (RuntimeException) exception; - } - return new RE(exception); - } - } - - /** - * {@link ForkJoinPool} friendly {@link Sequence} to {@link OrderedResultBatch} {@link Yielder} - */ - private static class SequenceYielder implements ForkJoinPool.ManagedBlocker - { - private final Sequence sequence; - private final int batchSize; - private volatile Yielder> batchYielder; - - public SequenceYielder(Sequence sequence, int batchSize) - { - this.sequence = sequence; - this.batchSize = batchSize; - } - - public Yielder> getYielder() - { - try { - ForkJoinPool.managedBlock(this); - return batchYielder; - } - catch (InterruptedException e) { - throw new RuntimeException("Failed to load initial batch of results", e); + outputQueue.offer(new ResultBatch<>()); } } - @Override - public boolean block() - { - batchYielder = OrderedResultBatch.fromSequence(sequence, batchSize); - return true; - } - - @Override - public boolean isReleasable() - { - return batchYielder != null; - } } @@ -812,19 +690,19 @@ public void offer(E item) * blocking that is needed to move results between stages of {@link MergeCombineAction} done in parallel, allowing * the fork join tasks to focus on doing actual work instead of dealing with managed blocking. */ - static class OrderedResultBatch + static class ResultBatch { @Nullable private final Queue values; private final boolean isTerminal; - OrderedResultBatch(int batchSize) + ResultBatch(int batchSize) { this.values = new ArrayDeque<>(batchSize); this.isTerminal = false; } - OrderedResultBatch() + ResultBatch() { this.values = null; this.isTerminal = true; @@ -861,15 +739,15 @@ public boolean isTerminalResult() /** * Convert sequence to yielder that accumulates results into ordered 'batches' */ - static Yielder> fromSequence(Sequence sequence, int batchSize) + static Yielder> fromSequence(Sequence sequence, int batchSize) { return sequence.toYielder( - new OrderedResultBatch<>(batchSize), - new YieldingAccumulator, E>() + new ResultBatch<>(batchSize), + new YieldingAccumulator, E>() { int count = 0; @Override - public OrderedResultBatch accumulate(OrderedResultBatch accumulated, E in) + public ResultBatch accumulate(ResultBatch accumulated, E in) { count++; if (count % batchSize == 0) { @@ -884,7 +762,46 @@ public OrderedResultBatch accumulate(OrderedResultBatch accumulated, E in) } /** - * Provides a higher level cursor interface to provide individual results out {@link OrderedResultBatch} provided by + * {@link ForkJoinPool} friendly conversion of {@link Sequence} to {@link Yielder< ResultBatch >} + */ + static class SequenceBatcher implements ForkJoinPool.ManagedBlocker + { + private final Sequence sequence; + private final int batchSize; + private volatile Yielder> batchYielder; + + public SequenceBatcher(Sequence sequence, int batchSize) + { + this.sequence = sequence; + this.batchSize = batchSize; + } + + public Yielder> getBatchYielder() + { + try { + ForkJoinPool.managedBlock(this); + return batchYielder; + } + catch (InterruptedException e) { + throw new RuntimeException("Failed to load initial batch of results", e); + } + } + @Override + public boolean block() + { + batchYielder = ResultBatch.fromSequence(sequence, batchSize); + return true; + } + + @Override + public boolean isReleasable() + { + return batchYielder != null; + } + } + + /** + * Provides a higher level cursor interface to provide individual results out {@link ResultBatch} provided by * a {@link Yielder} or {@link BlockingQueue}. This is the mechanism that powers {@link MergeCombineAction}, where * a set of {@link BatchedResultsCursor} are placed in a {@link PriorityQueue} to facilitate ordering to merge results * from these cursors, and combine results with the same ordering using the combining function. @@ -893,18 +810,14 @@ abstract static class BatchedResultsCursor implements ForkJoinPool.ManagedBlocker, Comparable> { final Ordering ordering; - volatile OrderedResultBatch resultBatch; + volatile ResultBatch resultBatch; BatchedResultsCursor(Ordering ordering) { this.ordering = ordering; } - public void initialize() - { - // nothing to initialize for yielders since they come primed, blocking queue will need to block for some data - // though so it is ready to go - } + public abstract void initialize(); public abstract void advance(); @@ -953,17 +866,24 @@ public int hashCode() } /** - * {@link BatchedResultsCursor} that wraps a {@link Yielder} of {@link OrderedResultBatch} to provide individual rows + * {@link BatchedResultsCursor} that wraps a {@link Yielder} of {@link ResultBatch} to provide individual rows * of the result batch. */ static class YielderBatchedResultsCursor extends BatchedResultsCursor { - Yielder> yielder; + final SequenceBatcher sequenceYielder; + Yielder> yielder; - YielderBatchedResultsCursor(Yielder> yielder, Ordering ordering) + YielderBatchedResultsCursor(SequenceBatcher sequenceYielder, Ordering ordering) { super(ordering); - this.yielder = yielder; + this.sequenceYielder = sequenceYielder; + } + + @Override + public void initialize() + { + yielder = sequenceYielder.getBatchYielder(); resultBatch = yielder.get(); } @@ -993,7 +913,7 @@ public boolean block() return true; } if (resultBatch == null || resultBatch.isDrained()) { - final Yielder> nextYielder = yielder.next(resultBatch); + final Yielder> nextYielder = yielder.next(resultBatch); yielder = nextYielder; } return true; @@ -1018,17 +938,17 @@ public void close() } /** - * {@link BatchedResultsCursor} that wraps a {@link BlockingQueue} of {@link OrderedResultBatch} to provide individual + * {@link BatchedResultsCursor} that wraps a {@link BlockingQueue} of {@link ResultBatch} to provide individual * rows from the result batch. */ static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor { - final BlockingQueue> queue; + final BlockingQueue> queue; final boolean hasTimeout; final long timeoutAt; BlockingQueueuBatchedResultsCursor( - BlockingQueue> blockingQueue, + BlockingQueue> blockingQueue, Ordering ordering, boolean hasTimeout, long timeoutAt @@ -1098,4 +1018,38 @@ public boolean isReleasable() return resultBatch != null; } } + + + /** + * Token to allow any {@link RecursiveAction} signal the others and the output sequence that something bad happened + * and processing should cancel, such as a timeout or connection loss. + */ + static class CancellationGizmo + { + // volatile instead of AtomicBoolean because it is never unset + private volatile boolean cancelled; + private volatile Exception exception; + + void cancel(Exception ex) + { + if (cancelled) { + return; + } + cancelled = true; + exception = ex; + } + + boolean isCancelled() + { + return cancelled; + } + + RuntimeException getRuntimeException() + { + if (exception instanceof RuntimeException) { + return (RuntimeException) exception; + } + return new RE(exception); + } + } } diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index cdc1e2f4bd0a..727a869c3e03 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -82,12 +82,12 @@ public void teardown() public void testOrderedResultBatchFromSequence() throws IOException { Sequence rawSequence = generateOrderedPairsSequence(5000); - Yielder> batchYielder = - ParallelMergeCombiningSequence.OrderedResultBatch.fromSequence(rawSequence, 128); - ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = - new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>(batchYielder, ordering); - + new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( + new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), + ordering + ); + cursor.initialize(); Yielder rawYielder = Yielders.each(rawSequence); IntPair prev = null; @@ -109,21 +109,21 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I final int batchSize = 128; final int sequenceSize = 5_000; Sequence rawSequence = generateOrderedPairsSequence(sequenceSize); - Yielder> batchYielder = - ParallelMergeCombiningSequence.OrderedResultBatch.fromSequence(rawSequence, batchSize); - ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = - new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>(batchYielder, ordering); - + new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( + new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), + ordering + ); + cursor.initialize(); Yielder rawYielder = Yielders.each(rawSequence); - ArrayBlockingQueue> outputQueue = + ArrayBlockingQueue> outputQueue = new ArrayBlockingQueue<>((int) Math.ceil(((double) sequenceSize / batchSize) + 2)); IntPair prev = null; - ParallelMergeCombiningSequence.OrderedResultBatch currentBatch = - new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + ParallelMergeCombiningSequence.ResultBatch currentBatch = + new ParallelMergeCombiningSequence.ResultBatch<>(batchSize); int batchCounter = 0; while (!rawYielder.isDone() && !cursor.isDone()) { Assert.assertEquals(rawYielder.get(), cursor.get()); @@ -134,7 +134,7 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I batchCounter++; if (batchCounter >= batchSize) { outputQueue.offer(currentBatch); - currentBatch = new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + currentBatch = new ParallelMergeCombiningSequence.ResultBatch<>(batchSize); batchCounter = 0; } rawYielder = rawYielder.next(rawYielder.get()); @@ -143,7 +143,7 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(new ParallelMergeCombiningSequence.OrderedResultBatch<>()); + outputQueue.offer(new ParallelMergeCombiningSequence.ResultBatch<>()); rawYielder.close(); cursor.close(); @@ -179,21 +179,22 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE final int batchSize = 128; final int sequenceSize = 5_000; Sequence rawSequence = generateOrderedPairsSequence(sequenceSize); - Yielder> batchYielder = - ParallelMergeCombiningSequence.OrderedResultBatch.fromSequence(rawSequence, batchSize); - ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = - new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>(batchYielder, ordering); + new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( + new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), + ordering + ); + cursor.initialize(); Yielder rawYielder = Yielders.each(rawSequence); - ArrayBlockingQueue> outputQueue = + ArrayBlockingQueue> outputQueue = new ArrayBlockingQueue<>((int) Math.ceil(((double) sequenceSize / batchSize) + 2)); IntPair prev = null; - ParallelMergeCombiningSequence.OrderedResultBatch currentBatch = - new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + ParallelMergeCombiningSequence.ResultBatch currentBatch = + new ParallelMergeCombiningSequence.ResultBatch<>(batchSize); int batchCounter = 0; while (!rawYielder.isDone() && !cursor.isDone()) { Assert.assertEquals(rawYielder.get(), cursor.get()); @@ -204,7 +205,7 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE batchCounter++; if (batchCounter >= batchSize) { outputQueue.offer(currentBatch); - currentBatch = new ParallelMergeCombiningSequence.OrderedResultBatch<>(batchSize); + currentBatch = new ParallelMergeCombiningSequence.ResultBatch<>(batchSize); batchCounter = 0; } rawYielder = rawYielder.next(rawYielder.get()); @@ -213,7 +214,7 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(new ParallelMergeCombiningSequence.OrderedResultBatch<>()); + outputQueue.offer(new ParallelMergeCombiningSequence.ResultBatch<>()); rawYielder.close(); cursor.close(); From 8ed849435bd212d511282e709776a557de11383d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 2 Oct 2019 13:38:24 -0700 Subject: [PATCH 19/51] formatting --- .../guava/ParallelMergeCombiningSequence.java | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index fb649a0c7a05..3e5f80463a29 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -201,12 +201,13 @@ public T next() @Override public void cleanup(Iterator iterFromMake) { - // todo: do ... something? + // 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. @@ -319,9 +320,7 @@ void spawnParallelTasks(int parallelMergeTasks) List>> partitions = Lists.partition(sequences, sequences.size() / parallelMergeTasks); - for (List> partition : partitions) { - BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); intermediaryOutputs.add(outputQueue); QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); @@ -476,7 +475,6 @@ protected void compute() } // else, push accumulated value to the queue, accumulate again with next value as initial - outputBatch.add(currentCombinedValue); batchCounter++; if (batchCounter >= batchSize) { @@ -540,6 +538,7 @@ protected void compute() } } + /** * This {@link RecursiveAction}, given a set of uninitialized {@link BatchedResultsCursor}, will initialize each of * them (which is a potentially managed blocking operation) so that each will produce a {@link ResultBatch} @@ -671,7 +670,6 @@ void addItem(E item) this.item = item; } - public void offer(E item) { try { @@ -726,12 +724,12 @@ public E next() return values.poll(); } - public boolean isDrained() + boolean isDrained() { return !isTerminal && values.isEmpty(); } - public boolean isTerminalResult() + boolean isTerminalResult() { return isTerminal; } @@ -761,6 +759,7 @@ public ResultBatch accumulate(ResultBatch accumulated, E in) } } + /** * {@link ForkJoinPool} friendly conversion of {@link Sequence} to {@link Yielder< ResultBatch >} */ @@ -770,13 +769,13 @@ static class SequenceBatcher implements ForkJoinPool.ManagedBlocker private final int batchSize; private volatile Yielder> batchYielder; - public SequenceBatcher(Sequence sequence, int batchSize) + SequenceBatcher(Sequence sequence, int batchSize) { this.sequence = sequence; this.batchSize = batchSize; } - public Yielder> getBatchYielder() + Yielder> getBatchYielder() { try { ForkJoinPool.managedBlock(this); @@ -786,6 +785,7 @@ public Yielder> getBatchYielder() throw new RuntimeException("Failed to load initial batch of results", e); } } + @Override public boolean block() { @@ -800,6 +800,7 @@ public boolean isReleasable() } } + /** * Provides a higher level cursor interface to provide individual results out {@link ResultBatch} provided by * a {@link Yielder} or {@link BlockingQueue}. This is the mechanism that powers {@link MergeCombineAction}, where @@ -821,7 +822,9 @@ abstract static class BatchedResultsCursor public abstract void advance(); - public void nextBatch() + public abstract boolean isDone(); + + void nextBatch() { try { ForkJoinPool.managedBlock(this); @@ -831,7 +834,6 @@ public void nextBatch() } } - public abstract boolean isDone(); public void close() { @@ -852,7 +854,7 @@ public int compareTo(BatchedResultsCursor o) @Override public boolean equals(Object o) { - if (!(o instanceof ParallelMergeCombiningSequence.BatchedResultsCursor)) { + if (!(o instanceof BatchedResultsCursor)) { return false; } return compareTo((BatchedResultsCursor) o) == 0; @@ -865,6 +867,7 @@ public int hashCode() } } + /** * {@link BatchedResultsCursor} that wraps a {@link Yielder} of {@link ResultBatch} to provide individual rows * of the result batch. @@ -937,6 +940,7 @@ public void close() } } + /** * {@link BatchedResultsCursor} that wraps a {@link BlockingQueue} of {@link ResultBatch} to provide individual * rows from the result batch. From 4301c0f677c727290a8cab1dd186b89225a3b5c7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 7 Oct 2019 11:38:37 -0700 Subject: [PATCH 20/51] pushdown config --- .../druid/query/DruidProcessingConfig.java | 22 +++++++++++++++++++ .../org/apache/druid/query/QueryContexts.java | 15 +++++-------- .../druid/client/CachingClusteredClient.java | 8 +++---- 3 files changed, 31 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index 8d3cc07d7afd..4ed5b0b4fa67 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -34,6 +34,8 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem public static final int DEFAULT_NUM_MERGE_BUFFERS = -1; public static final int DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = -1; public static final int MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024; + public static final int DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS = 1024; + public static final int DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS = 128; public static final int DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS = 60_000; private AtomicReference computedBufferSizeBytes = new AtomicReference<>(); @@ -158,6 +160,26 @@ public int getNumThreadsMergePoolConfigured() return DEFAULT_NUM_THREADS; } + + + @Config(value = "${base_path}.mergePoolDefaultMaxParallelism") + public int getMergePoolDefaultMaxParallelism() + { + return Integer.MAX_VALUE; + } + + @Config(value = "${base_path}.mergePoolTaskInitialYieldRows") + public int getMergePoolTaskInitialYieldRows() + { + return DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS; + } + + @Config(value = "${base_path}.mergePoolSmallBatchRows") + public int getMergePoolSmallBatchRows() + { + return DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS; + } + public int getNumThreadsMergePool() { int numThreadsConfigured = getNumThreadsMergePoolConfigured(); diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 88e29cec3614..538bee1e98eb 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -57,9 +57,6 @@ public class QueryContexts public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); public static final long NO_TIMEOUT = 0; public static final boolean DEFAULT_ENABLE_PARALLEL_MERGE = true; - public static final int DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS = 1024; - public static final int DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS = 128; - public static final int DEFAULT_PARALLEL_MERGE_PARALLELISM = Integer.MAX_VALUE; @SuppressWarnings("unused") // Used by Jackson serialization public enum Vectorize @@ -193,19 +190,19 @@ public static boolean getEnableParallelMerges(Query query) return parseBoolean(query, BROKER_PARALLEL_MERGE_KEY, DEFAULT_ENABLE_PARALLEL_MERGE); } - public static int getParallelMergeInitialYieldRows(Query query) + public static int getParallelMergeInitialYieldRows(Query query, int defaultValue) { - return parseInt(query, BROKER_PARALLEL_MERGE_INITIAL_YIELD_ROWS_KEY, DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS); + return parseInt(query, BROKER_PARALLEL_MERGE_INITIAL_YIELD_ROWS_KEY, defaultValue); } - public static int getParallelMergeSmallBatchRows(Query query) + public static int getParallelMergeSmallBatchRows(Query query, int defaultValue) { - return parseInt(query, BROKER_PARALLEL_MERGE_SMALL_BATCH_ROWS_KEY, DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS); + return parseInt(query, BROKER_PARALLEL_MERGE_SMALL_BATCH_ROWS_KEY, defaultValue); } - public static int getParallelMergeParallelism(Query query) + public static int getParallelMergeParallelism(Query query, int defaultValue) { - return parseInt(query, BROKER_PARALLELISM, DEFAULT_PARALLEL_MERGE_PARALLELISM); + return parseInt(query, BROKER_PARALLELISM, defaultValue); } @Deprecated diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 9aac1114adca..3bf935ebc955 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -305,8 +305,6 @@ private Sequence merge(List> sequencesByInterval) { BinaryOperator mergeFn = toolChest.createMergeFn(query); if (processingConfig.useParallelMergePool() && QueryContexts.getEnableParallelMerges(query) && mergeFn != null) { - final int yieldAfter = QueryContexts.getParallelMergeInitialYieldRows(query); - final int batchSize = QueryContexts.getParallelMergeSmallBatchRows(query); return new ParallelMergeCombiningSequence<>( pool, sequencesByInterval, @@ -315,9 +313,9 @@ private Sequence merge(List> sequencesByInterval) QueryContexts.hasTimeout(query), QueryContexts.getTimeout(query), QueryContexts.getPriority(query), - QueryContexts.getParallelMergeParallelism(query), - yieldAfter, - batchSize + QueryContexts.getParallelMergeParallelism(query, processingConfig.getMergePoolDefaultMaxParallelism()), + QueryContexts.getParallelMergeInitialYieldRows(query, processingConfig.getMergePoolTaskInitialYieldRows()), + QueryContexts.getParallelMergeSmallBatchRows(query, processingConfig.getMergePoolSmallBatchRows()) ); } else { return Sequences From 5e543574a17780a915c2c4d1e1497fd35573b2ef Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 10 Oct 2019 01:02:01 -0700 Subject: [PATCH 21/51] use nanos consistently, move logs back to debug level, bit more javadoc --- .../guava/ParallelMergeCombiningSequence.java | 94 ++++++++++++------- 1 file changed, 62 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 3e5f80463a29..3a7f71fe1a44 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -57,7 +57,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final BinaryOperator combineFn; private final int queueSize; private final boolean hasTimeout; - private final long timeoutAt; + private final long timeoutAtNanos; private final int queryPriority; // not currently used :( private final int yieldAfter; private final int batchSize; @@ -82,7 +82,7 @@ public ParallelMergeCombiningSequence( this.orderingFn = orderingFn; this.combineFn = combineFn; this.hasTimeout = hasTimeout; - this.timeoutAt = System.currentTimeMillis() + timeoutMillis; + this.timeoutAtNanos = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutMillis, TimeUnit.MILLISECONDS); this.queryPriority = queryPriority; this.parallelism = parallelism; this.yieldAfter = yieldAfter; @@ -109,11 +109,11 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat yieldAfter, batchSize, hasTimeout, - timeoutAt, + timeoutAtNanos, cancellationGizmo ); workerPool.execute(finalMergeAction); - Sequence finalOutSequence = makeOutputSequenceForQueue(outputQueue, hasTimeout, timeoutAt, cancellationGizmo); + Sequence finalOutSequence = makeOutputSequenceForQueue(outputQueue, hasTimeout, timeoutAtNanos, cancellationGizmo); return finalOutSequence.toYielder(initValue, accumulator); } @@ -124,7 +124,7 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat static Sequence makeOutputSequenceForQueue( BlockingQueue> queue, boolean hasTimeout, - long timeoutAt, + long timeoutAtNanos, CancellationGizmo cancellationGizmo ) { @@ -141,8 +141,8 @@ public Iterator make() @Override public boolean hasNext() { - final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); - if (thisTimeout < 0) { + final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); + if (thisTimeoutNanos < 0) { throw new RE(new TimeoutException("Sequence iterator timed out")); } @@ -152,7 +152,7 @@ public boolean hasNext() try { if (currentBatch == null || currentBatch.isDrained()) { if (hasTimeout) { - currentBatch = queue.poll(thisTimeout, TimeUnit.MILLISECONDS); + currentBatch = queue.poll(thisTimeoutNanos, TimeUnit.NANOSECONDS); } else { currentBatch = queue.take(); } @@ -207,7 +207,6 @@ public void cleanup(Iterator iterFromMake) ); } - /** * 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. @@ -277,7 +276,7 @@ protected void compute() // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, // just serially perform the merge-combine with a single task if (sequences.size() < 4 || parallelTaskCount < 2) { - LOG.info( + LOG.debug( "Input sequence count (%s) or available parallel merge task count (%s) too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", sequences.size(), @@ -302,7 +301,7 @@ protected void compute() getPool().execute(blockForInputsAction); } else { // 2 layer parallel merge done in fjp - LOG.info("Spawning %s parallel merge-combine tasks for %s sequences", parallelTaskCount, sequences.size()); + LOG.debug("Spawning %s parallel merge-combine tasks for %s sequences", parallelTaskCount, sequences.size()); spawnParallelTasks(parallelTaskCount); } } @@ -372,10 +371,13 @@ void spawnParallelTasks(int parallelMergeTasks) */ int computeNumTasks() { + final int availableProcessors = JvmUtils.getRuntimeInfo().getAvailableProcessors(); + final int runningThreadCount = getPool().getRunningThreadCount(); + final int submissionCount = getPool().getQueuedSubmissionCount(); // max is minimum of either number of processors or user suggested parallelism - final int maxParallelism = Math.min(JvmUtils.getRuntimeInfo().getAvailableProcessors(), parallelism); + final int maxParallelism = Math.min(availableProcessors, parallelism); // adjust max to be no more than total pool parallelism less the number of running threads + submitted tasks - final int utilizationEstimate = getPool().getRunningThreadCount() + getPool().getQueuedSubmissionCount(); + final int utilizationEstimate = runningThreadCount + submissionCount; // minimum of 'max computed parallelism' and pool parallelism less current 'utilization estimate' final int computedParallelism = Math.min(maxParallelism, getPool().getParallelism() - utilizationEstimate); // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot @@ -383,7 +385,24 @@ int computeNumTasks() (int) Math.floor((double) sequences.size() / 2.0), computedParallelism - 1 ); - return computedOptimalParallelism; + + final int computedNumParallelTasks = Math.max(computedOptimalParallelism, 1); + if (LOG.isDebugEnabled()) { + LOG.debug("Computed parallel tasks: [%s]; ForkJoinPool details - processors: [%s] parallelism: [%s] " + + "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] " + + "pool size: [%s] steal count: [%s]", + computedNumParallelTasks, + availableProcessors, + parallelism, + getPool().getActiveThreadCount(), + runningThreadCount, + submissionCount, + getPool().getQueuedTaskCount(), + getPool().getPoolSize(), + getPool().getStealCount() + ); + } + return computedNumParallelTasks; } } @@ -400,7 +419,9 @@ int computeNumTasks() * task completes and executes a new task to continue where it left off. This value is initially set by the * {@link MergeCombinePartitioningAction} to a default value, but after that this process is timed to try and compute * an 'optimal' number of rows to yield to achieve a task runtime of ~10ms, on the assumption that the time to process - * n results will be approximately the same. + * n results will be approximately the same. 'recursionDepth' is used to track how many times a task has continued + * executing, and utilized to compute a cumulative moving average of task run time per amount yielded in order to + * 'smooth' out the continual adjustment. */ private static class MergeCombineAction extends RecursiveAction { @@ -411,7 +432,7 @@ private static class MergeCombineAction extends RecursiveAction private final T initialValue; private final int yieldAfter; private final int batchSize; - private final int depth; + private final int recursionDepth; private final CancellationGizmo cancellationGizmo; private MergeCombineAction( @@ -422,7 +443,7 @@ private MergeCombineAction( T initialValue, int yieldAfter, int batchSize, - int depth, + int recursionDepth, CancellationGizmo cancellationGizmo ) { @@ -433,7 +454,7 @@ private MergeCombineAction( this.initialValue = initialValue; this.yieldAfter = yieldAfter; this.batchSize = batchSize; - this.depth = depth; + this.recursionDepth = recursionDepth; this.cancellationGizmo = cancellationGizmo; } @@ -505,9 +526,16 @@ protected void compute() 1L ); final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); - final double cumulativeMovingAverage = (nextYieldAfter + (depth * yieldAfter)) / (depth + 1); + final double cumulativeMovingAverage = (nextYieldAfter + (recursionDepth * yieldAfter)) / (recursionDepth + 1); final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); + LOG.debug( + "task %s yielded %s results ran for %s millis, next task yielding every %s operations", + recursionDepth, + yieldAfter, + elapsedMillis, + nextYieldAfter + ); getPool().execute(new MergeCombineAction<>( pQueue, outputQueue, @@ -516,12 +544,13 @@ protected void compute() currentCombinedValue, adjustedNextYieldAfter, batchSize, - depth + 1, + recursionDepth + 1, cancellationGizmo )); } else if (cancellationGizmo.isCancelled()) { // if we got the cancellation signal, go ahead and write terminal value into output queue to help gracefully // allow downstream stuff to stop + LOG.debug("cancelled after %s tasks", recursionDepth); outputQueue.offer(new ResultBatch<>()); } else { // if priority queue is empty, push the final accumulated value into the output batch and push it out @@ -529,6 +558,7 @@ protected void compute() outputQueue.offer(outputBatch); // ... and the terminal value to indicate the blocking queue holding the values is complete outputQueue.offer(new ResultBatch<>()); + LOG.debug("merge combine complete after %s tasks", recursionDepth); } } catch (Exception ex) { @@ -627,15 +657,15 @@ protected void compute() static class QueuePusher implements ForkJoinPool.ManagedBlocker { final boolean hasTimeout; - final long timeoutAt; + final long timeoutAtNanos; final BlockingQueue queue; volatile E item = null; - QueuePusher(BlockingQueue q, boolean hasTimeout, long timeoutAt) + QueuePusher(BlockingQueue q, boolean hasTimeout, long timeoutAtNanos) { this.queue = q; this.hasTimeout = hasTimeout; - this.timeoutAt = timeoutAt; + this.timeoutAtNanos = timeoutAtNanos; } @Override @@ -644,11 +674,11 @@ public boolean block() throws InterruptedException boolean success = false; if (item != null) { if (hasTimeout) { - final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); - if (thisTimeout < 0) { + final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); + if (thisTimeoutNanos < 0) { throw new RE(new TimeoutException("QueuePusher timed out offering data")); } - success = queue.offer(item, thisTimeout, TimeUnit.MILLISECONDS); + success = queue.offer(item, thisTimeoutNanos, TimeUnit.NANOSECONDS); } else { success = queue.offer(item); } @@ -949,19 +979,19 @@ static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor< { final BlockingQueue> queue; final boolean hasTimeout; - final long timeoutAt; + final long timeoutAtNanos; BlockingQueueuBatchedResultsCursor( BlockingQueue> blockingQueue, Ordering ordering, boolean hasTimeout, - long timeoutAt + long timeoutAtNanos ) { super(ordering); this.queue = blockingQueue; this.hasTimeout = hasTimeout; - this.timeoutAt = timeoutAt; + this.timeoutAtNanos = timeoutAtNanos; } @Override @@ -998,11 +1028,11 @@ public boolean block() throws InterruptedException { if (resultBatch == null || resultBatch.isDrained()) { if (hasTimeout) { - final int thisTimeout = Ints.checkedCast(timeoutAt - System.currentTimeMillis()); - if (thisTimeout < 0) { + final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); + if (thisTimeoutNanos < 0) { throw new RE(new TimeoutException("BlockingQueue cursor timed out waiting for data")); } - resultBatch = queue.poll(thisTimeout, TimeUnit.MILLISECONDS); + resultBatch = queue.poll(thisTimeoutNanos, TimeUnit.NANOSECONDS); } else { resultBatch = queue.take(); } From 8c02cc5f174e43b0520896b814b479eae1ff6453 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 10 Oct 2019 01:09:15 -0700 Subject: [PATCH 22/51] static terminal result batch --- .../guava/ParallelMergeCombiningSequence.java | 16 +++++++++------- .../ParallelMergeCombiningSequenceTest.java | 7 ++++--- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 3a7f71fe1a44..1a60a23ed14c 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -307,7 +307,7 @@ protected void compute() } catch (Exception ex) { cancellationGizmo.cancel(ex); - out.offer(new ResultBatch<>()); + out.offer(ResultBatch.TERMINAL); } } @@ -551,19 +551,19 @@ protected void compute() // if we got the cancellation signal, go ahead and write terminal value into output queue to help gracefully // allow downstream stuff to stop LOG.debug("cancelled after %s tasks", recursionDepth); - outputQueue.offer(new ResultBatch<>()); + outputQueue.offer(ResultBatch.TERMINAL); } else { // if priority queue is empty, push the final accumulated value into the output batch and push it out outputBatch.add(currentCombinedValue); outputQueue.offer(outputBatch); // ... and the terminal value to indicate the blocking queue holding the values is complete - outputQueue.offer(new ResultBatch<>()); + outputQueue.offer(ResultBatch.TERMINAL); LOG.debug("merge combine complete after %s tasks", recursionDepth); } } catch (Exception ex) { cancellationGizmo.cancel(ex); - outputQueue.offer(new ResultBatch<>()); + outputQueue.offer(ResultBatch.TERMINAL); } } } @@ -638,12 +638,12 @@ protected void compute() cancellationGizmo )); } else { - outputQueue.offer(new ResultBatch<>()); + outputQueue.offer(ResultBatch.TERMINAL); } } catch (Exception ex) { cancellationGizmo.cancel(ex); - outputQueue.offer(new ResultBatch<>()); + outputQueue.offer(ResultBatch.TERMINAL); } } } @@ -720,6 +720,8 @@ public void offer(E item) */ static class ResultBatch { + static final ResultBatch TERMINAL = new ResultBatch(); + @Nullable private final Queue values; private final boolean isTerminal; @@ -730,7 +732,7 @@ static class ResultBatch this.isTerminal = false; } - ResultBatch() + private ResultBatch() { this.values = null; this.isTerminal = true; diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index 727a869c3e03..b02888bbd68a 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -39,6 +39,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.BinaryOperator; @@ -143,7 +144,7 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(new ParallelMergeCombiningSequence.ResultBatch<>()); + outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.TERMINAL); rawYielder.close(); cursor.close(); @@ -153,7 +154,7 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I Sequence queueAsSequence = ParallelMergeCombiningSequence.makeOutputSequenceForQueue( outputQueue, true, - System.currentTimeMillis() + 10_000, + System.nanoTime() + TimeUnit.NANOSECONDS.convert(10_000, TimeUnit.MILLISECONDS), new ParallelMergeCombiningSequence.CancellationGizmo() ); @@ -214,7 +215,7 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(new ParallelMergeCombiningSequence.ResultBatch<>()); + outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.TERMINAL); rawYielder.close(); cursor.close(); From 9336289d5c4402492be56b500653409980a066be Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 10 Oct 2019 01:31:38 -0700 Subject: [PATCH 23/51] javadoc for nullability of createMergeFn --- .../main/java/org/apache/druid/query/QueryToolChest.java | 7 ++++++- .../org/apache/druid/query/ResultMergeQueryRunner.java | 3 +++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index 511af9f4b3b6..f30a4c528df0 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -111,7 +111,12 @@ public QueryRunner mergeResults(QueryRunner runner) /** * Creates a merge function that is used to merge intermediate aggregates from historicals in broker. This merge * function is used in the default {@link ResultMergeQueryRunner} provided by - * {@link QueryToolChest#mergeResults(QueryRunner)} and can be used in additional future merge implementations + * {@link QueryToolChest#mergeResults(QueryRunner)} and also used in + * {@link org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence} by 'CachingClusteredClient' if it + * does not return null. + * + * Returning null from this function means that a query does not support result merging, at + * least via the mechanisms that utilize this function. */ @Nullable public BinaryOperator createMergeFn(Query query) diff --git a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java index 6361479d24ef..1a01f7e1cc14 100644 --- a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java @@ -19,6 +19,7 @@ package org.apache.druid.query; +import com.google.common.base.Preconditions; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.guava.Sequence; @@ -43,6 +44,8 @@ public ResultMergeQueryRunner( ) { super(baseRunner); + Preconditions.checkNotNull(comparatorGenerator); + Preconditions.checkNotNull(mergeFnGenerator); this.comparatorGenerator = comparatorGenerator; this.mergeFnGenerator = mergeFnGenerator; } From 23b7a37f55dad0c0a17149d5ff7866476977530a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 10 Oct 2019 01:36:50 -0700 Subject: [PATCH 24/51] cleanup --- .../guava/ParallelMergeCombiningSequence.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 1a60a23ed14c..488a53398bf5 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -387,21 +387,21 @@ int computeNumTasks() ); final int computedNumParallelTasks = Math.max(computedOptimalParallelism, 1); - if (LOG.isDebugEnabled()) { - LOG.debug("Computed parallel tasks: [%s]; ForkJoinPool details - processors: [%s] parallelism: [%s] " - + "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] " - + "pool size: [%s] steal count: [%s]", - computedNumParallelTasks, - availableProcessors, - parallelism, - getPool().getActiveThreadCount(), - runningThreadCount, - submissionCount, - getPool().getQueuedTaskCount(), - getPool().getPoolSize(), - getPool().getStealCount() - ); - } + + LOG.debug("Computed parallel tasks: [%s]; ForkJoinPool details - processors: [%s] parallelism: [%s] " + + "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] " + + "pool size: [%s] steal count: [%s]", + computedNumParallelTasks, + availableProcessors, + parallelism, + getPool().getActiveThreadCount(), + runningThreadCount, + submissionCount, + getPool().getQueuedTaskCount(), + getPool().getPoolSize(), + getPool().getStealCount() + ); + return computedNumParallelTasks; } } From ef80b53081931f9c0a9fa4751d04f598e0dc5130 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 10 Oct 2019 13:16:23 -0700 Subject: [PATCH 25/51] oops --- .../java/util/common/guava/ParallelMergeCombiningSequence.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 488a53398bf5..7c059cd6f3b7 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -21,7 +21,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import com.google.common.primitives.Ints; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.utils.JvmUtils; @@ -401,7 +400,7 @@ int computeNumTasks() getPool().getPoolSize(), getPool().getStealCount() ); - + return computedNumParallelTasks; } } From 7097f22c5882391e7ea763caf938255e617b7db8 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 10 Oct 2019 18:53:56 -0700 Subject: [PATCH 26/51] fix race, add docs --- .../guava/ParallelMergeCombiningSequence.java | 20 +++++------- docs/configuration/index.md | 6 ++++ docs/querying/query-context.md | 5 +++ .../druid/query/DruidProcessingConfig.java | 31 +++++++++---------- 4 files changed, 34 insertions(+), 28 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 7c059cd6f3b7..37c7bb291110 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -41,6 +41,7 @@ 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; /** @@ -1061,30 +1062,25 @@ public boolean isReleasable() */ static class CancellationGizmo { - // volatile instead of AtomicBoolean because it is never unset - private volatile boolean cancelled; - private volatile Exception exception; + private final AtomicReference exception = new AtomicReference<>(null); void cancel(Exception ex) { - if (cancelled) { - return; - } - cancelled = true; - exception = ex; + exception.compareAndSet(null, ex); } boolean isCancelled() { - return cancelled; + return exception.get() != null; } RuntimeException getRuntimeException() { - if (exception instanceof RuntimeException) { - return (RuntimeException) exception; + Exception ex = exception.get(); + if (ex instanceof RuntimeException) { + return (RuntimeException) ex; } - return new RE(exception); + return new RE(ex); } } } diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 3d0e7f6a90ae..c31e2f1b997a 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1435,6 +1435,12 @@ The broker uses processing configs for nested groupBy queries. And, if you use g |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| +|`druid.processing.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`| +|`druid.processing.numMergePoolThreads`|Size of ForkJoinPool|`druid.processing.numThreads * 1.5`| +|`druid.processing.mergePoolAwaitShutdownMillis`|Time to wait for merge ForkJoinPool tasks to complete before ungracefully stopping on process shutdown in milliseconds.|`60_000`| +|`druid.processing.mergePoolDefaultMaxParallelism`|Default per query maximum number of parallel merge tasks.|`unlimited`| +|`druid.processing.mergePoolTaskInitialYieldRows`|Number of rows to yield per ForkJoinPool merge task, before forking off a new task to continue merging sequences.|`1024`| +|`druid.processing.mergePoolSmallBatchRows`|Size of result batches to operate on in ForkJoinPool merge tasks.|`128`| The amount of direct memory needed by Druid is at least `druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md index a3c8c1a4db71..067a333a8a7f 100644 --- a/docs/querying/query-context.md +++ b/docs/querying/query-context.md @@ -41,6 +41,11 @@ The query context is used for various query configuration parameters. The follow |maxQueuedBytes | `druid.broker.http.maxQueuedBytes` | Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled.| |serializeDateTimeAsLong| `false` | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute process| |serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between Broker and compute process| +|enableParallelMerge|`true`|Enable parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| +|parallelMergeParallelism|`druid.processing.numMergePoolThreads`|Maximum number of parallel threads to use for parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| +|parallelMergeInitialYieldRows|`druid.processing.mergePoolTaskInitialYieldRows`|Number of rows to yield per ForkJoinPool merge task for parallel result merging on the Broker, before forking off a new task to continue merging sequences. See [Broker configuration](../configuration/index.html#broker) for more details.| +|parallelMergeSmallBatchRows|`druid.processing.mergePoolTaskInitialYieldRows`|Size of result batches to operate on in ForkJoinPool merge tasks for parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| + In addition, some query types offer context parameters specific to that query type. diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index 4ed5b0b4fa67..56fc7f2dcd1f 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -160,7 +160,21 @@ public int getNumThreadsMergePoolConfigured() return DEFAULT_NUM_THREADS; } + public int getNumThreadsMergePool() + { + int numThreadsConfigured = getNumThreadsMergePoolConfigured(); + if (numThreadsConfigured != DEFAULT_NUM_THREADS) { + return numThreadsConfigured; + } else { + return (int) Math.ceil(JvmUtils.getRuntimeInfo().getAvailableProcessors() * 1.5); + } + } + @Config(value = "${base_path}.mergePoolAwaitShutdownMillis") + public long getMergePoolAwaitShutdownMillis() + { + return DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS; + } @Config(value = "${base_path}.mergePoolDefaultMaxParallelism") public int getMergePoolDefaultMaxParallelism() @@ -179,20 +193,5 @@ public int getMergePoolSmallBatchRows() { return DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS; } - - public int getNumThreadsMergePool() - { - int numThreadsConfigured = getNumThreadsMergePoolConfigured(); - if (numThreadsConfigured != DEFAULT_NUM_THREADS) { - return numThreadsConfigured; - } else { - return (int) Math.ceil(JvmUtils.getRuntimeInfo().getAvailableProcessors() * 1.5); - } - } - - @Config(value = "${base_path}.mergePoolAwaitShutdownMillis") - public long getMergePoolAwaitShutdownMillis() - { - return DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS; - } } + From bd0710cc3423125b0fece45865cb78bc6e02dc77 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 11 Oct 2019 02:59:39 -0700 Subject: [PATCH 27/51] spelling, remove todo, add unhandled exception log --- docs/configuration/index.md | 2 +- .../java/org/apache/druid/guice/DruidProcessingModule.java | 4 ++-- website/.spelling | 5 +++++ 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index c31e2f1b997a..8f7826c46abe 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1436,7 +1436,7 @@ The broker uses processing configs for nested groupBy queries. And, if you use g |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| |`druid.processing.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`| -|`druid.processing.numMergePoolThreads`|Size of ForkJoinPool|`druid.processing.numThreads * 1.5`| +|`druid.processing.numMergePoolThreads`|Size of ForkJoinPool|`druid.processing.numThreads * 1.5` (rounded up)| |`druid.processing.mergePoolAwaitShutdownMillis`|Time to wait for merge ForkJoinPool tasks to complete before ungracefully stopping on process shutdown in milliseconds.|`60_000`| |`druid.processing.mergePoolDefaultMaxParallelism`|Default per query maximum number of parallel merge tasks.|`unlimited`| |`druid.processing.mergePoolTaskInitialYieldRows`|Number of rows to yield per ForkJoinPool merge task, before forking off a new task to continue merging sequences.|`1024`| diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index fcc5026e9deb..8c24b0548efc 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -142,8 +142,8 @@ public LifecycleForkJoinPoolProvider getMergeProcessingPoolProvider(DruidProcess { return new LifecycleForkJoinPoolProvider( config.getNumThreadsMergePool(), - ForkJoinPool.defaultForkJoinWorkerThreadFactory, // todo: ? - null, + ForkJoinPool.defaultForkJoinWorkerThreadFactory, + (t, e) -> log.error(e, "Unhandled exception in thread [%s]", t), true, config.getMergePoolAwaitShutdownMillis() ); diff --git a/website/.spelling b/website/.spelling index cd9ece12b5ce..c963ecb8cd6d 100644 --- a/website/.spelling +++ b/website/.spelling @@ -164,6 +164,7 @@ aggregator aggregators ambari analytics +async authorizer authorizers autocomplete @@ -1264,10 +1265,14 @@ druid.broker.cache.useCache druid.broker.cache.useResultLevelCache druid.historical.cache.populateCache druid.historical.cache.useCache +enableParallelMerge floatSum maxQueuedBytes maxScatterGatherBytes minTopNThreshold +parallelMergeInitialYieldRows +parallelMergeParallelism +parallelMergeSmallBatchRows populateCache populateResultLevelCache queryId From 024d8902a9b9025c4f85ef137c3c91a754091f8d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 11 Oct 2019 03:18:33 -0700 Subject: [PATCH 28/51] cleanup, revert unintended change --- .../druid/common/guava/CombiningSequence.java | 2 +- .../guava/ParallelMergeCombiningSequence.java | 13 ++----------- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java index bc4119b2b5b2..9e9a7d77df57 100644 --- a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java +++ b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java @@ -227,7 +227,7 @@ boolean accumulatedSomething() } } - class CombiningAccumulator implements Accumulator + private class CombiningAccumulator implements Accumulator { private OutType retVal; private final Accumulator accumulator; diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 37c7bb291110..724b20c62fe0 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -162,11 +162,7 @@ public boolean hasNext() } if (cancellationGizmo.isCancelled()) { - if (cancellationGizmo.getRuntimeException() != null) { - throw cancellationGizmo.getRuntimeException(); - } else { - throw new RuntimeException("Failed to merge results, unknown error"); - } + throw cancellationGizmo.getRuntimeException(); } if (currentBatch.isTerminalResult()) { @@ -183,11 +179,7 @@ public boolean hasNext() public T next() { if (cancellationGizmo.isCancelled()) { - if (cancellationGizmo.getRuntimeException() != null) { - throw cancellationGizmo.getRuntimeException(); - } else { - throw new RuntimeException("Failed to merge results, unknown error"); - } + throw cancellationGizmo.getRuntimeException(); } if (currentBatch == null || currentBatch.isDrained() || currentBatch.isTerminalResult()) { @@ -866,7 +858,6 @@ void nextBatch() } } - public void close() { // nothing to close for blocking queue, but yielders will need to clean up or they will leak resources From 9cbe4d095953b00e53f397b86441348260c3050f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 11 Oct 2019 03:21:24 -0700 Subject: [PATCH 29/51] another unintended change --- .../java/org/apache/druid/common/guava/ComplexSequenceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java index c6cf2cb3266a..ba5504f25cf1 100644 --- a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java +++ b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java @@ -36,7 +36,7 @@ public class ComplexSequenceTest { // Integer::sum with more nulls - public static final BinaryOperator PLUS_NULLABLE = (arg1, arg2) -> { + private static final BinaryOperator PLUS_NULLABLE = (arg1, arg2) -> { if (arg1 == null) { return arg2; } From 206566ebdd927aea8a21df59e7bd2caf7794c673 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 11 Oct 2019 18:51:21 -0700 Subject: [PATCH 30/51] review stuff --- .../guava/ParallelMergeCombiningSequence.java | 42 ++++++++----------- 1 file changed, 18 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 724b20c62fe0..f89ba0674c56 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -204,7 +204,7 @@ public void cleanup(Iterator iterFromMake) * 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 + * for each partition a {@link PrepareMergeCombineInputsAction} will be executed 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 @@ -303,7 +303,7 @@ protected void compute() } } - void spawnParallelTasks(int parallelMergeTasks) + private void spawnParallelTasks(int parallelMergeTasks) { List tasks = new ArrayList<>(); List>> intermediaryOutputs = new ArrayList<>(parallelMergeTasks); @@ -361,7 +361,7 @@ void spawnParallelTasks(int parallelMergeTasks) * {@link ForkJoinPool} utilization. A return value of 1 or less indicates that a serial merge will be done on * the pool instead. */ - int computeNumTasks() + private int computeNumTasks() { final int availableProcessors = JvmUtils.getRuntimeInfo().getAvailableProcessors(); final int runningThreadCount = getPool().getRunningThreadCount(); @@ -373,6 +373,8 @@ int computeNumTasks() // minimum of 'max computed parallelism' and pool parallelism less current 'utilization estimate' final int computedParallelism = Math.min(maxParallelism, getPool().getParallelism() - utilizationEstimate); // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot + // we divide the sequences by 2 because we need at least 2 sequences per partition for it to make sense to need + // an additional parallel task to compute the merge final int computedOptimalParallelism = Math.min( (int) Math.floor((double) sequences.size() / 2.0), computedParallelism - 1 @@ -407,13 +409,13 @@ int computeNumTasks() * order for a {@link PriorityQueue}, and as a comparison to determine if 'same' ordered results need to be combined * with a supplied {@link BinaryOperator} combining function. * - * This task takes a 'yieldAfter' parameter which controls how many input result rows will be processed before this - * task completes and executes a new task to continue where it left off. This value is initially set by the + * This task takes a {@link #yieldAfter} parameter which controls how many input result rows will be processed before + * this task completes and executes a new task to continue where it left off. This value is initially set by the * {@link MergeCombinePartitioningAction} to a default value, but after that this process is timed to try and compute * an 'optimal' number of rows to yield to achieve a task runtime of ~10ms, on the assumption that the time to process - * n results will be approximately the same. 'recursionDepth' is used to track how many times a task has continued - * executing, and utilized to compute a cumulative moving average of task run time per amount yielded in order to - * 'smooth' out the continual adjustment. + * n results will be approximately the same. {@link #recursionDepth} is used to track how many times a task has + * continued executing, and utilized to compute a cumulative moving average of task run time per amount yielded in + * order to 'smooth' out the continual adjustment. */ private static class MergeCombineAction extends RecursiveAction { @@ -687,15 +689,10 @@ public boolean isReleasable() return item == null; } - void addItem(E item) - { - this.item = item; - } - public void offer(E item) { try { - addItem(item); + this.item = item; ForkJoinPool.managedBlock(this); } catch (InterruptedException e) { @@ -716,18 +713,15 @@ static class ResultBatch @Nullable private final Queue values; - private final boolean isTerminal; ResultBatch(int batchSize) { this.values = new ArrayDeque<>(batchSize); - this.isTerminal = false; } private ResultBatch() { this.values = null; - this.isTerminal = true; } public void add(E in) @@ -750,12 +744,12 @@ public E next() boolean isDrained() { - return !isTerminal && values.isEmpty(); + return values != null && values.isEmpty(); } boolean isTerminalResult() { - return isTerminal; + return values == null; } /** @@ -771,11 +765,11 @@ static Yielder> fromSequence(Sequence sequence, int batchS @Override public ResultBatch accumulate(ResultBatch accumulated, E in) { + accumulated.add(in); count++; if (count % batchSize == 0) { yield(); } - accumulated.add(in); return accumulated; } } @@ -897,19 +891,19 @@ public int hashCode() */ static class YielderBatchedResultsCursor extends BatchedResultsCursor { - final SequenceBatcher sequenceYielder; + final SequenceBatcher batcher; Yielder> yielder; - YielderBatchedResultsCursor(SequenceBatcher sequenceYielder, Ordering ordering) + YielderBatchedResultsCursor(SequenceBatcher batcher, Ordering ordering) { super(ordering); - this.sequenceYielder = sequenceYielder; + this.batcher = batcher; } @Override public void initialize() { - yielder = sequenceYielder.getBatchYielder(); + yielder = batcher.getBatchYielder(); resultBatch = yielder.get(); } From a179dca14eda5cb8bf60384d098f82a54c742751 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 18 Oct 2019 17:52:03 -0700 Subject: [PATCH 31/51] add ParallelMergeCombiningSequenceBenchmark, fixes --- benchmarks/pom.xml | 6 + ...rallelMergeCombiningSequenceBenchmark.java | 246 ++++++++++++++++++ .../guava/ParallelMergeCombiningSequence.java | 69 +++-- .../ParallelMergeCombiningSequenceTest.java | 140 +++++++--- .../druid/query/DruidProcessingConfig.java | 29 ++- .../druid/client/CachingClusteredClient.java | 5 +- .../druid/guice/DruidProcessingModule.java | 2 +- 7 files changed, 423 insertions(+), 74 deletions(-) create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 2c6e9ebda0f3..4c3502d53bd1 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -81,6 +81,12 @@ druid-histogram ${project.parent.version} + + org.apache.druid + druid-core + ${project.parent.version} + test-jar + org.apache.druid druid-processing diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java new file mode 100644 index 000000000000..93ccf346a68c --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java @@ -0,0 +1,246 @@ +/* + * 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.benchmark; + +import org.apache.druid.common.guava.CombiningSequence; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.MergeSequence; +import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence; +import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequenceTest; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.logger.Logger; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + + +@State(Scope.Benchmark) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +@Warmup(iterations = 10) +@Measurement(iterations = 30) +public class ParallelMergeCombiningSequenceBenchmark +{ + private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); + private static final ForkJoinPool mergePool = new ForkJoinPool( + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + ForkJoinPool.defaultForkJoinWorkerThreadFactory, + (t, e) -> log.error(e, "Unhandled exception in thread [%s]", t), + true + ); + + private static final ExecutorService consumer = Execs.multiThreaded(40, "mock-http-thread"); + + @Param({"8", "16", "32", "64"}) + private int numSequences; + + @Param({"75000"}) + private int rowsPerSequence; + + @Param({"1", "2", "4", "8", "16", "32", "64"}) + private int concurrentSequenceConsumers; + + @Param({ + "combiningMergeSequence-same-thread", + "parallelism-1-10ms-256-1024", + "parallelism-4-10ms-256-1024", + "parallelism-8-10ms-256-1024", + "parallelism-16-10ms-256-1024", + "parallelism-32-10ms-256-1024", + "parallelism-1-100ms-512-4096", + "parallelism-4-100ms-512-4096", + "parallelism-8-100ms-512-4096", + "parallelism-16-100ms-512-4096", + "parallelism-32-100ms-512-4096", + "parallelism-1-100ms-1024-4096", + "parallelism-4-100ms-1024-4096", + "parallelism-8-100ms-1024-4096", + "parallelism-16-100ms-1024-4096", + "parallelism-32-100ms-1024-4096", + "parallelism-1-100ms-1024-16384", + "parallelism-4-100ms-1024-16384", + "parallelism-8-100ms-1024-16384", + "parallelism-16-100ms-1024-16384", + "parallelism-32-100ms-1024-16384", + "parallelism-1-100ms-4096-16384", + "parallelism-4-100ms-4096-16384", + "parallelism-8-100ms-4096-16384", + "parallelism-16-100ms-4096-16384", + "parallelism-32-100ms-4096-16384", + }) + private String strategy; + + @Param({ + "non-blocking", + "initial-block-random-500ms", + "initial-block-random-5000ms", + "slow-sequence-random-100ms-1ms" + }) + private String inputSequenceType; + + private List> inputSequences; + + private int parallelism; + private int targetTaskTimeMillis; + private int batchSize; + private int yieldAfter; + + private Supplier> inputSequenceFactory; + private Supplier> outputSequenceFactory; + + @Setup(Level.Trial) + public void setup() + { + String[] inputSequenceTypeSplit = inputSequenceType.split("-"); + if ("initial".equals(inputSequenceTypeSplit[0])) { + // e.g. "initial-block-random-{startDelay}ms" + final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[3].substring(0, inputSequenceTypeSplit[3].length() - 2)); + inputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.initialDelaySequence(rowsPerSequence, delayMillis); + } else if("slow".equals(inputSequenceTypeSplit[0])) { + // e.g. "slow-sequence-random-{startDelay}ms-{frequencyDelay}ms" + final int startDelayMillis = Integer.parseInt(inputSequenceTypeSplit[3].substring(0,inputSequenceTypeSplit[3].length() - 2)); + final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); + final int frequency = rowsPerSequence / 10; + inputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.slowSequence(rowsPerSequence, frequency, startDelayMillis, delayMillis); + } else { // non-blocking sequence + inputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.generateOrderedPairsSequence(rowsPerSequence); + } + + inputSequences = new ArrayList<>(numSequences); + for (int i = 0; i < numSequences; i++) { + inputSequences.add(inputSequenceFactory.get()); + } + + String[] strategySplit = strategy.split("-"); + if ("parallelism".equals(strategySplit[0])) { + // "parallelism-{parallelism}-{targetTime}ms-{batchSize}-{yieldAfter}" + parallelism = Integer.parseInt(strategySplit[1]); + targetTaskTimeMillis = Integer.parseInt(strategySplit[2].substring(0,strategySplit[2].length() - 2)); + batchSize = Integer.parseInt(strategySplit[3]); + yieldAfter = Integer.parseInt(strategySplit[4]); + outputSequenceFactory = () -> createParallelSequence(); + } else { + outputSequenceFactory = () -> createCombiningMergeSequence(); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void exec(Blackhole blackhole) throws Exception + { + List futures = new ArrayList<>(concurrentSequenceConsumers); + for (int i = 0; i < concurrentSequenceConsumers; i++) { + futures.add( + consumer.submit(() -> { + try { + consumeSequence(blackhole); + } catch (Exception anyException) { + log.error(anyException, "benchmark failed"); + } + }) + ); + } + + for (int i = 0; i < concurrentSequenceConsumers; i++) { + blackhole.consume(futures.get(i).get()); + } + blackhole.consume(futures); + } + + private Sequence createParallelSequence() + { + return new ParallelMergeCombiningSequence<>( + mergePool, + inputSequences, + ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, + ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN, + false, + 0, + 0, + parallelism, + yieldAfter, + batchSize, + targetTaskTimeMillis + ); + } + + private Sequence createCombiningMergeSequence() + { + return CombiningSequence.create( + new MergeSequence<>(ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, Sequences.simple(inputSequences)), + ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, + ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN + ); + } + + private void consumeSequence(Blackhole blackhole) + { + Yielder yielder = Yielders.each(outputSequenceFactory.get()); + + ParallelMergeCombiningSequenceTest.IntPair prev; + while (!yielder.isDone()) { + prev = yielder.get(); + blackhole.consume(prev); + yielder = yielder.next(yielder.get()); + } + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ParallelMergeCombiningSequenceBenchmark.class.getSimpleName()) + .forks(1) + .syncIterations(true) + .resultFormat(ResultFormatType.CSV) + .result("parallel-merge-combining-sequence.csv") + .build(); + + new Runner(opt).run(); + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index f89ba0674c56..eb04f6c0504d 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -45,12 +45,22 @@ import java.util.function.BinaryOperator; /** - * Artisanal, locally-sourced, hand-crafted, gluten and GMO free, bespoke, small-batch parallel merge combinining sequence + * Artisanal, locally-sourced, hand-crafted, gluten and GMO free, bespoke, free-range, organic, small-batch parallel + * merge combining sequence. + * + * See proposal: https://github.com/apache/incubator-druid/issues/8577 + * + * Functionally equivalent to wrapping {@link org.apache.druid.common.guava.CombiningSequence} around a + * {@link MergeSequence}, but done in parallel on a {@link ForkJoinPool} running in 'async' mode. */ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { private static final Logger LOG = new Logger(ParallelMergeCombiningSequence.class); + public static final int DEFAULT_TASK_TARGET_RUN_TIME_MILLIS = 10; + public static final int DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS = 1024; + public static final int DEFAULT_TASK_SMALL_BATCH_NUM_ROWS = 128; + private final ForkJoinPool workerPool; private final List> baseSequences; private final Ordering orderingFn; @@ -62,6 +72,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final int yieldAfter; private final int batchSize; private final int parallelism; + private final long targetTimeNanos; private final CancellationGizmo cancellationGizmo; public ParallelMergeCombiningSequence( @@ -74,7 +85,8 @@ public ParallelMergeCombiningSequence( int queryPriority, int parallelism, int yieldAfter, - int batchSize + int batchSize, + int targetTimeMillis ) { this.workerPool = workerPool; @@ -87,6 +99,7 @@ public ParallelMergeCombiningSequence( this.parallelism = parallelism; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.targetTimeNanos = TimeUnit.NANOSECONDS.convert(targetTimeMillis, TimeUnit.MILLISECONDS); this.queueSize = 4 * (yieldAfter / batchSize); this.cancellationGizmo = new CancellationGizmo(); } @@ -108,6 +121,7 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat parallelism, yieldAfter, batchSize, + targetTimeNanos, hasTimeout, timeoutAtNanos, cancellationGizmo @@ -142,7 +156,7 @@ public Iterator make() public boolean hasNext() { final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); - if (thisTimeoutNanos < 0) { + if (hasTimeout && thisTimeoutNanos < 0) { throw new RE(new TimeoutException("Sequence iterator timed out")); } @@ -228,6 +242,7 @@ private static class MergeCombinePartitioningAction extends RecursiveAction private final int parallelism; private final int yieldAfter; private final int batchSize; + private final long targetTimeNanos; private final boolean hasTimeout; private final long timeoutAt; private final CancellationGizmo cancellationGizmo; @@ -241,6 +256,7 @@ private MergeCombinePartitioningAction( int parallelism, int yieldAfter, int batchSize, + long targetTimeNanos, boolean hasTimeout, long timeoutAt, CancellationGizmo cancellationGizmo @@ -254,6 +270,7 @@ private MergeCombinePartitioningAction( this.parallelism = parallelism; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.targetTimeNanos = targetTimeNanos; this.hasTimeout = hasTimeout; this.timeoutAt = timeoutAt; this.cancellationGizmo = cancellationGizmo; @@ -267,7 +284,7 @@ protected void compute() // if we have a small number of sequences to merge, or computed paralellism is too low, do not run in parallel, // just serially perform the merge-combine with a single task - if (sequences.size() < 4 || parallelTaskCount < 2) { + if (parallelTaskCount < 2) { LOG.debug( "Input sequence count (%s) or available parallel merge task count (%s) too small to perform parallel" + " merge-combine, performing serially with a single merge-combine task", @@ -288,6 +305,7 @@ protected void compute() combineFn, yieldAfter, batchSize, + targetTimeNanos, cancellationGizmo ); getPool().execute(blockForInputsAction); @@ -327,6 +345,7 @@ private void spawnParallelTasks(int parallelMergeTasks) combineFn, yieldAfter, batchSize, + targetTimeNanos, cancellationGizmo ); tasks.add(blockForInputsAction); @@ -350,6 +369,7 @@ private void spawnParallelTasks(int parallelMergeTasks) combineFn, yieldAfter, batchSize, + targetTimeNanos, cancellationGizmo ); @@ -367,14 +387,16 @@ private int computeNumTasks() final int runningThreadCount = getPool().getRunningThreadCount(); final int submissionCount = getPool().getQueuedSubmissionCount(); // max is minimum of either number of processors or user suggested parallelism - final int maxParallelism = Math.min(availableProcessors, parallelism); + final int maxParallelism = Math.min(Math.min(availableProcessors, parallelism), getPool().getParallelism()); // adjust max to be no more than total pool parallelism less the number of running threads + submitted tasks - final int utilizationEstimate = runningThreadCount + submissionCount; - // minimum of 'max computed parallelism' and pool parallelism less current 'utilization estimate' - final int computedParallelism = Math.min(maxParallelism, getPool().getParallelism() - utilizationEstimate); + // minus 1 for the task that is running this calculation since it will be replaced with the parallel tasks + final int utilizationEstimate = runningThreadCount + submissionCount - 1; + // 'computed parallelism' is the remaineder of the 'max parallelism' less current 'utilization estimate' + final int computedParallelism = maxParallelism - utilizationEstimate; // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot // we divide the sequences by 2 because we need at least 2 sequences per partition for it to make sense to need - // an additional parallel task to compute the merge + // an additional parallel task to compute the merge, so if we have a small number of total sequences this might be + // below final int computedOptimalParallelism = Math.min( (int) Math.floor((double) sequences.size() / 2.0), computedParallelism - 1 @@ -426,6 +448,7 @@ private static class MergeCombineAction extends RecursiveAction private final T initialValue; private final int yieldAfter; private final int batchSize; + private final long targetTimeNanos; private final int recursionDepth; private final CancellationGizmo cancellationGizmo; @@ -437,6 +460,7 @@ private MergeCombineAction( T initialValue, int yieldAfter, int batchSize, + long targetTimeNanos, int recursionDepth, CancellationGizmo cancellationGizmo ) @@ -448,6 +472,7 @@ private MergeCombineAction( this.initialValue = initialValue; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.targetTimeNanos = targetTimeNanos; this.recursionDepth = recursionDepth; this.cancellationGizmo = cancellationGizmo; } @@ -457,6 +482,7 @@ protected void compute() { try { long start = System.nanoTime(); + long startCpuNanos = JvmUtils.safeGetThreadCpuTime(); int counter = 0; int batchCounter = 0; @@ -515,20 +541,21 @@ protected void compute() // measure the time it took to process 'yieldAfter' elements in order to project a next 'yieldAfter' value // which we want to target a 10ms task run time. smooth this value with a cumulative moving average in order // to prevent normal jitter in processing time from skewing the next yield value too far in any direction - final long elapsedMillis = Math.max( - TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS), - 1L - ); - final double nextYieldAfter = Math.max(10.0 * ((double) yieldAfter / elapsedMillis), 1.0); - final double cumulativeMovingAverage = (nextYieldAfter + (recursionDepth * yieldAfter)) / (recursionDepth + 1); + final long elapsedNanos = System.nanoTime() - start; + final long elapsedCpuNanos = JvmUtils.safeGetThreadCpuTime() - startCpuNanos; + final double nextYieldAfter = Math.max((double) targetTimeNanos * ((double) yieldAfter / elapsedCpuNanos), 1.0); + final double cumulativeMovingAverage = + (nextYieldAfter + (recursionDepth * yieldAfter)) / (recursionDepth + 1); final int adjustedNextYieldAfter = (int) Math.ceil(cumulativeMovingAverage); LOG.debug( - "task %s yielded %s results ran for %s millis, next task yielding every %s operations", + "task recursion %s yielded %s results ran for %s millis (%s nanos), %s cpu nanos, next task yielding every %s operations", recursionDepth, yieldAfter, - elapsedMillis, - nextYieldAfter + TimeUnit.MILLISECONDS.convert(elapsedNanos, TimeUnit.NANOSECONDS), + elapsedNanos, + elapsedCpuNanos, + adjustedNextYieldAfter ); getPool().execute(new MergeCombineAction<>( pQueue, @@ -538,6 +565,7 @@ protected void compute() currentCombinedValue, adjustedNextYieldAfter, batchSize, + targetTimeNanos, recursionDepth + 1, cancellationGizmo )); @@ -585,6 +613,7 @@ private static class PrepareMergeCombineInputsAction extends RecursiveAction private final QueuePusher> outputQueue; private final int yieldAfter; private final int batchSize; + private final long targetTimeNanos; private final CancellationGizmo cancellationGizmo; private PrepareMergeCombineInputsAction( @@ -594,6 +623,7 @@ private PrepareMergeCombineInputsAction( BinaryOperator combineFn, int yieldAfter, int batchSize, + long targetTimeNanos, CancellationGizmo cancellationGizmo ) { @@ -603,6 +633,7 @@ private PrepareMergeCombineInputsAction( this.outputQueue = outputQueue; this.yieldAfter = yieldAfter; this.batchSize = batchSize; + this.targetTimeNanos = targetTimeNanos; this.cancellationGizmo = cancellationGizmo; } @@ -628,6 +659,7 @@ protected void compute() null, yieldAfter, batchSize, + targetTimeNanos, 1, cancellationGizmo )); @@ -933,6 +965,7 @@ public boolean block() return true; } if (resultBatch == null || resultBatch.isDrained()) { + resultBatch = new ResultBatch<>(batcher.batchSize); final Yielder> nextYielder = yielder.next(resultBatch); yielder = nextYielder; } diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index b02888bbd68a..d7a06c6bb479 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -46,13 +46,9 @@ public class ParallelMergeCombiningSequenceTest { private static final Logger LOG = new Logger(ParallelMergeCombiningSequenceTest.class); - private static final int DEFAULT_TEST_YIELD_AFTER = 1024; - private static final int DEFAULT_TEST_BATCH_SIZE = 128; - private ForkJoinPool pool; - - private final Ordering ordering = Ordering.natural().onResultOf(p -> p.lhs); - private final BinaryOperator mergeFn = (lhs, rhs) -> { + public static final Ordering INT_PAIR_ORDERING = Ordering.natural().onResultOf(p -> p.lhs); + public static final BinaryOperator INT_PAIR_MERGE_FN = (lhs, rhs) -> { if (lhs == null) { return rhs; } @@ -64,10 +60,17 @@ public class ParallelMergeCombiningSequenceTest return new IntPair(lhs.lhs, lhs.rhs + rhs.rhs); }; + private ForkJoinPool pool; + @Before public void setup() { - pool = new ForkJoinPool(4, ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true); + pool = new ForkJoinPool( + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + ForkJoinPool.defaultForkJoinWorkerThreadFactory, + (t, e) -> LOG.error(e, "Unhandled exception in thread [%s]", t), + true + ); } @After @@ -86,7 +89,7 @@ public void testOrderedResultBatchFromSequence() throws IOException ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), - ordering + INT_PAIR_ORDERING ); cursor.initialize(); Yielder rawYielder = Yielders.each(rawSequence); @@ -94,7 +97,6 @@ public void testOrderedResultBatchFromSequence() throws IOException IntPair prev = null; while (!rawYielder.isDone() && !cursor.isDone()) { Assert.assertEquals(rawYielder.get(), cursor.get()); - LOG.debug("%s", cursor.get()); Assert.assertNotEquals(cursor.get(), prev); prev = cursor.get(); rawYielder = rawYielder.next(rawYielder.get()); @@ -113,7 +115,7 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), - ordering + INT_PAIR_ORDERING ); cursor.initialize(); @@ -128,7 +130,6 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I int batchCounter = 0; while (!rawYielder.isDone() && !cursor.isDone()) { Assert.assertEquals(rawYielder.get(), cursor.get()); - LOG.debug("%s", cursor.get()); Assert.assertNotEquals(cursor.get(), prev); prev = cursor.get(); currentBatch.add(prev); @@ -160,10 +161,8 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I Yielder queueYielder = Yielders.each(queueAsSequence); - int rowCtr = 0; while (!rawYielder.isDone() && !queueYielder.isDone()) { Assert.assertEquals(rawYielder.get(), queueYielder.get()); - LOG.debug("row %s: %s", rowCtr++, queueYielder.get()); Assert.assertNotEquals(queueYielder.get(), prev); prev = queueYielder.get(); rawYielder = rawYielder.next(rawYielder.get()); @@ -183,7 +182,7 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), - ordering + INT_PAIR_ORDERING ); cursor.initialize(); @@ -199,7 +198,6 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE int batchCounter = 0; while (!rawYielder.isDone() && !cursor.isDone()) { Assert.assertEquals(rawYielder.get(), cursor.get()); - LOG.debug("%s", cursor.get()); Assert.assertNotEquals(cursor.get(), prev); prev = cursor.get(); currentBatch.add(prev); @@ -225,7 +223,7 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor queueCursor = new ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor<>( outputQueue, - ordering, + INT_PAIR_ORDERING, false, -1L ); @@ -233,7 +231,6 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE prev = null; while (!rawYielder.isDone() && !queueCursor.isDone()) { Assert.assertEquals(rawYielder.get(), queueCursor.get()); - LOG.debug("%s", queueCursor.get()); Assert.assertNotEquals(queueCursor.get(), prev); prev = queueCursor.get(); rawYielder = rawYielder.next(rawYielder.get()); @@ -451,13 +448,19 @@ public void testTimeoutExceptionDueToStalledInput() throws Exception input.add(generateOrderedPairsSequence(someSize)); input.add(generateOrderedPairsSequence(someSize)); input.add(generateOrderedPairsSequence(someSize)); - input.add(slowSequence(someSize, 500)); + input.add(slowSequence(someSize, 1, 500, 500)); expectedException.expect(RuntimeException.class); expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); expectedException.expectMessage("Sequence iterator timed out waiting for data"); - assertException(input, DEFAULT_TEST_BATCH_SIZE, DEFAULT_TEST_YIELD_AFTER, 1000L, 0); + assertException( + input, + ParallelMergeCombiningSequence.DEFAULT_TASK_SMALL_BATCH_NUM_ROWS, + ParallelMergeCombiningSequence.DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS, + 1000L, + 0 + ); } @Test @@ -478,29 +481,34 @@ public void testTimeoutExceptionDueToStalledReader() throws Exception private void assertResult(List> sequences) throws InterruptedException, IOException { - assertResult(sequences, DEFAULT_TEST_BATCH_SIZE, DEFAULT_TEST_YIELD_AFTER); + assertResult( + sequences, + ParallelMergeCombiningSequence.DEFAULT_TASK_SMALL_BATCH_NUM_ROWS, + ParallelMergeCombiningSequence.DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS + ); } private void assertResult(List> sequences, int batchSize, int yieldAfter) throws InterruptedException, IOException { final CombiningSequence combiningSequence = CombiningSequence.create( - new MergeSequence<>(ordering, Sequences.simple(sequences)), - ordering, - mergeFn + new MergeSequence<>(INT_PAIR_ORDERING, Sequences.simple(sequences)), + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN ); final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( pool, sequences, - ordering, - mergeFn, + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN, true, 5000, 0, Runtime.getRuntime().availableProcessors() - 1, yieldAfter, - batchSize + batchSize, + ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS ); Yielder combiningYielder = Yielders.each(combiningSequence); @@ -510,7 +518,6 @@ private void assertResult(List> sequences, int batchSize, int while (!combiningYielder.isDone() && !parallelMergeCombineYielder.isDone()) { Assert.assertEquals(combiningYielder.get(), parallelMergeCombineYielder.get()); - LOG.debug("%s", parallelMergeCombineYielder.get()); Assert.assertNotEquals(parallelMergeCombineYielder.get(), prev); prev = parallelMergeCombineYielder.get(); combiningYielder = combiningYielder.next(combiningYielder.get()); @@ -529,7 +536,13 @@ private void assertResult(List> sequences, int batchSize, int private void assertException(List> sequences) throws Exception { - assertException(sequences, DEFAULT_TEST_BATCH_SIZE, DEFAULT_TEST_YIELD_AFTER, 5000L, 0); + assertException( + sequences, + ParallelMergeCombiningSequence.DEFAULT_TASK_SMALL_BATCH_NUM_ROWS, + ParallelMergeCombiningSequence.DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS, + 5000L, + 0 + ); } private void assertException( @@ -545,14 +558,15 @@ private void assertException( final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( pool, sequences, - ordering, - mergeFn, + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN, true, timeout, 0, Runtime.getRuntime().availableProcessors() - 1, yieldAfter, - batchSize + batchSize, + ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS ); Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); @@ -560,7 +574,6 @@ private void assertException( IntPair prev = null; while (!parallelMergeCombineYielder.isDone()) { - LOG.debug("%s", parallelMergeCombineYielder.get()); Assert.assertNotEquals(parallelMergeCombineYielder.get(), prev); prev = parallelMergeCombineYielder.get(); if (readDelayMillis > 0 && ThreadLocalRandom.current().nextBoolean()) { @@ -571,20 +584,20 @@ private void assertException( parallelMergeCombineYielder.close(); } catch (Exception ex) { - LOG.warn(ex, "actual exception:"); + LOG.warn(ex, "exception:"); throw ex; } } - static class IntPair extends Pair + public static class IntPair extends Pair { - IntPair(@Nullable Integer lhs, @Nullable Integer rhs) + public IntPair(@Nullable Integer lhs, @Nullable Integer rhs) { super(lhs, rhs); } } - private static List generateOrderedPairs(int length) + public static List generateOrderedPairs(int length) { int counter = 0; int i = 0; @@ -599,12 +612,12 @@ private static List generateOrderedPairs(int length) return generatedSequence; } - private static Sequence generateOrderedPairsSequence(int length) + public static Sequence generateOrderedPairsSequence(int length) { return Sequences.simple(generateOrderedPairs(length)); } - private static Sequence explodingSequence(int explodeAfter) + public static Sequence explodingSequence(int explodeAfter) { List items = generateOrderedPairs(explodeAfter + 1); return new BaseSequence<>( @@ -642,7 +655,50 @@ public void cleanup(Iterator iterFromMake) ); } - private static Sequence slowSequence(int size, int delay) + public static Sequence initialDelaySequence(int size, int maxDelayMillis) + { + List items = generateOrderedPairs(size); + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + int i = 0; + @Override + public boolean hasNext() + { + return i < items.size(); + } + + @Override + public IntPair next() + { + if (i == 0) { + try { + Thread.sleep(ThreadLocalRandom.current().nextInt(1, maxDelayMillis)); + } + catch (InterruptedException ex) { + throw new RuntimeException(ex); + } + } + return items.get(i++); + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } + } + ); + } + + public static Sequence slowSequence(int size, int frequency, int maxStartDelayMillis, int maxDelayMillis) { List items = generateOrderedPairs(size); return new BaseSequence<>( @@ -663,9 +719,11 @@ public boolean hasNext() @Override public IntPair next() { - if (ThreadLocalRandom.current().nextBoolean()) { + if (i == 0 || (i % frequency == 0 && ThreadLocalRandom.current().nextBoolean())) { try { - Thread.sleep(delay); + Thread.sleep( + ThreadLocalRandom.current().nextInt(1, i == 0 ? maxStartDelayMillis : maxDelayMillis) + ); } catch (InterruptedException ex) { throw new RuntimeException(ex); diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index 56fc7f2dcd1f..c774e6e2ec62 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -20,6 +20,7 @@ package org.apache.druid.query; import org.apache.druid.java.util.common.concurrent.ExecutorServiceConfig; +import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.utils.JvmUtils; @@ -34,8 +35,6 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem public static final int DEFAULT_NUM_MERGE_BUFFERS = -1; public static final int DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = -1; public static final int MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024; - public static final int DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS = 1024; - public static final int DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS = 128; public static final int DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS = 60_000; private AtomicReference computedBufferSizeBytes = new AtomicReference<>(); @@ -148,19 +147,19 @@ public String getTmpDir() return System.getProperty("java.io.tmpdir"); } - @Config(value = "${base_path}.useParallelMergePool") + @Config(value = "${base_path}.merge.useParallelMergePool") public boolean useParallelMergePool() { return true; } - @Config(value = "${base_path}.numMergePoolThreads") + @Config(value = "${base_path}.merge.pool.parallelism") public int getNumThreadsMergePoolConfigured() { return DEFAULT_NUM_THREADS; } - public int getNumThreadsMergePool() + public int getMergePoolParallelism() { int numThreadsConfigured = getNumThreadsMergePoolConfigured(); if (numThreadsConfigured != DEFAULT_NUM_THREADS) { @@ -170,28 +169,34 @@ public int getNumThreadsMergePool() } } - @Config(value = "${base_path}.mergePoolAwaitShutdownMillis") + @Config(value = "${base_path}.merge.pool.awaitShutdownMillis") public long getMergePoolAwaitShutdownMillis() { return DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS; } - @Config(value = "${base_path}.mergePoolDefaultMaxParallelism") - public int getMergePoolDefaultMaxParallelism() + @Config(value = "${base_path}.merge.pool.defaultMaxQueryParallelism") + public int getMergePoolDefaultMaxQueryParallelism() { return Integer.MAX_VALUE; } - @Config(value = "${base_path}.mergePoolTaskInitialYieldRows") + @Config(value = "${base_path}.merge.task.targetRunTimeMillis") + public int getMergePoolTargetTaskRunTimeMillis() + { + return ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS; + } + + @Config(value = "${base_path}.merge.task.initialYieldNumRows") public int getMergePoolTaskInitialYieldRows() { - return DEFAULT_PARALLEL_MERGE_INITIAL_YIELD_ROWS; + return ParallelMergeCombiningSequence.DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS; } - @Config(value = "${base_path}.mergePoolSmallBatchRows") + @Config(value = "${base_path}.merge.task.smallBatchNumRows") public int getMergePoolSmallBatchRows() { - return DEFAULT_PARALLEL_MERGE_SMALL_BATCH_ROWS; + return ParallelMergeCombiningSequence.DEFAULT_TASK_SMALL_BATCH_NUM_ROWS; } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 3bf935ebc955..b91731a54cef 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -313,9 +313,10 @@ private Sequence merge(List> sequencesByInterval) QueryContexts.hasTimeout(query), QueryContexts.getTimeout(query), QueryContexts.getPriority(query), - QueryContexts.getParallelMergeParallelism(query, processingConfig.getMergePoolDefaultMaxParallelism()), + QueryContexts.getParallelMergeParallelism(query, processingConfig.getMergePoolDefaultMaxQueryParallelism()), QueryContexts.getParallelMergeInitialYieldRows(query, processingConfig.getMergePoolTaskInitialYieldRows()), - QueryContexts.getParallelMergeSmallBatchRows(query, processingConfig.getMergePoolSmallBatchRows()) + QueryContexts.getParallelMergeSmallBatchRows(query, processingConfig.getMergePoolSmallBatchRows()), + processingConfig.getMergePoolTargetTaskRunTimeMillis() ); } else { return Sequences diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index 8c24b0548efc..59700160090a 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -141,7 +141,7 @@ public BlockingPool getMergeBufferPool(DruidProcessingConfig config) public LifecycleForkJoinPoolProvider getMergeProcessingPoolProvider(DruidProcessingConfig config) { return new LifecycleForkJoinPoolProvider( - config.getNumThreadsMergePool(), + config.getMergePoolParallelism(), ForkJoinPool.defaultForkJoinWorkerThreadFactory, (t, e) -> log.error(e, "Unhandled exception in thread [%s]", t), true, From 0f6cffc09b33e3f020c8f735d0091c5c55d0554c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 18 Oct 2019 21:28:03 -0700 Subject: [PATCH 32/51] hyper-threading is the enemy --- ...rallelMergeCombiningSequenceBenchmark.java | 31 +++++++++++++------ .../CachingClusteredClientBenchmark.java | 2 +- .../guava/ParallelMergeCombiningSequence.java | 12 +++---- .../ParallelMergeCombiningSequenceTest.java | 8 ++--- .../druid/query/DruidProcessingConfig.java | 6 ++-- 5 files changed, 36 insertions(+), 23 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java index 93ccf346a68c..9cb5a471cedf 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java @@ -64,22 +64,38 @@ public class ParallelMergeCombiningSequenceBenchmark { private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); + // default merge FJP size private static final ForkJoinPool mergePool = new ForkJoinPool( - (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.75), ForkJoinPool.defaultForkJoinWorkerThreadFactory, (t, e) -> log.error(e, "Unhandled exception in thread [%s]", t), true ); - private static final ExecutorService consumer = Execs.multiThreaded(40, "mock-http-thread"); + // this should be as large as the largest value of concurrentSequenceConsumers + private static final ExecutorService consumer = Execs.multiThreaded(64, "mock-http-thread"); - @Param({"8", "16", "32", "64"}) + // note: parameters are broken down to allow easily commenting out lines to mix and match which benchmarks to run + @Param({ + "8", + "16", + "32", + "64" + }) private int numSequences; @Param({"75000"}) private int rowsPerSequence; - @Param({"1", "2", "4", "8", "16", "32", "64"}) + @Param({ + "1", + "2", + "4", + "8", + "16", + "32", + "64" + }) private int concurrentSequenceConsumers; @Param({ @@ -88,27 +104,22 @@ public class ParallelMergeCombiningSequenceBenchmark "parallelism-4-10ms-256-1024", "parallelism-8-10ms-256-1024", "parallelism-16-10ms-256-1024", - "parallelism-32-10ms-256-1024", "parallelism-1-100ms-512-4096", "parallelism-4-100ms-512-4096", "parallelism-8-100ms-512-4096", "parallelism-16-100ms-512-4096", - "parallelism-32-100ms-512-4096", "parallelism-1-100ms-1024-4096", "parallelism-4-100ms-1024-4096", "parallelism-8-100ms-1024-4096", "parallelism-16-100ms-1024-4096", - "parallelism-32-100ms-1024-4096", "parallelism-1-100ms-1024-16384", "parallelism-4-100ms-1024-16384", "parallelism-8-100ms-1024-16384", "parallelism-16-100ms-1024-16384", - "parallelism-32-100ms-1024-16384", "parallelism-1-100ms-4096-16384", "parallelism-4-100ms-4096-16384", "parallelism-8-100ms-4096-16384", - "parallelism-16-100ms-4096-16384", - "parallelism-32-100ms-4096-16384", + "parallelism-16-100ms-4096-16384" }) private String strategy; diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 6e4768946570..4b26cfdf1b62 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -312,7 +312,7 @@ public > QueryToolChest getToolChest processingPool = Execs.multiThreaded(processingConfig.getNumThreads(), "caching-clustered-client-benchmark"); forkJoinPool = new ForkJoinPool( - (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.75), ForkJoinPool.defaultForkJoinWorkerThreadFactory, null, true diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index eb04f6c0504d..7850abb93ac2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -383,11 +383,11 @@ private void spawnParallelTasks(int parallelMergeTasks) */ private int computeNumTasks() { - final int availableProcessors = JvmUtils.getRuntimeInfo().getAvailableProcessors(); final int runningThreadCount = getPool().getRunningThreadCount(); final int submissionCount = getPool().getQueuedSubmissionCount(); - // max is minimum of either number of processors or user suggested parallelism - final int maxParallelism = Math.min(Math.min(availableProcessors, parallelism), getPool().getParallelism()); + // max is smaller of either parallelism passed into sequence (number of physical cores by default) or the pool + // parallelism + final int maxParallelism = Math.min(parallelism, getPool().getParallelism()); // adjust max to be no more than total pool parallelism less the number of running threads + submitted tasks // minus 1 for the task that is running this calculation since it will be replaced with the parallel tasks final int utilizationEstimate = runningThreadCount + submissionCount - 1; @@ -404,16 +404,16 @@ private int computeNumTasks() final int computedNumParallelTasks = Math.max(computedOptimalParallelism, 1); - LOG.debug("Computed parallel tasks: [%s]; ForkJoinPool details - processors: [%s] parallelism: [%s] " + LOG.debug("Computed parallel tasks: [%s]; ForkJoinPool details - sequence parallelism: [%s] " + "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] " - + "pool size: [%s] steal count: [%s]", + + "pool parallelism: [%s] pool size: [%s] steal count: [%s]", computedNumParallelTasks, - availableProcessors, parallelism, getPool().getActiveThreadCount(), runningThreadCount, submissionCount, getPool().getQueuedTaskCount(), + getPool().getParallelism(), getPool().getPoolSize(), getPool().getStealCount() ); diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index d7a06c6bb479..e7f9b9ad6eda 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -66,7 +66,7 @@ public class ParallelMergeCombiningSequenceTest public void setup() { pool = new ForkJoinPool( - (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 1.5), + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.75), ForkJoinPool.defaultForkJoinWorkerThreadFactory, (t, e) -> LOG.error(e, "Unhandled exception in thread [%s]", t), true @@ -505,7 +505,7 @@ private void assertResult(List> sequences, int batchSize, int true, 5000, 0, - Runtime.getRuntime().availableProcessors() - 1, + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.5), yieldAfter, batchSize, ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS @@ -563,7 +563,7 @@ private void assertException( true, timeout, 0, - Runtime.getRuntime().availableProcessors() - 1, + (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.5), yieldAfter, batchSize, ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS @@ -722,7 +722,7 @@ public IntPair next() if (i == 0 || (i % frequency == 0 && ThreadLocalRandom.current().nextBoolean())) { try { Thread.sleep( - ThreadLocalRandom.current().nextInt(1, i == 0 ? maxStartDelayMillis : maxDelayMillis) + ThreadLocalRandom.current().nextInt(1, i == 0 ? maxStartDelayMillis : maxDelayMillis) ); } catch (InterruptedException ex) { diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index c774e6e2ec62..8e827ff510fa 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -165,7 +165,8 @@ public int getMergePoolParallelism() if (numThreadsConfigured != DEFAULT_NUM_THREADS) { return numThreadsConfigured; } else { - return (int) Math.ceil(JvmUtils.getRuntimeInfo().getAvailableProcessors() * 1.5); + // assume 2 hyper-threads per core, so that this value is probably by default the number of physical cores * 1.5 + return (int) Math.ceil(JvmUtils.getRuntimeInfo().getAvailableProcessors() * 0.75); } } @@ -178,7 +179,8 @@ public long getMergePoolAwaitShutdownMillis() @Config(value = "${base_path}.merge.pool.defaultMaxQueryParallelism") public int getMergePoolDefaultMaxQueryParallelism() { - return Integer.MAX_VALUE; + // assume 2 hyper-threads per core, so that this value is probably by default the number of physical cores + return (int) Math.max(JvmUtils.getRuntimeInfo().getAvailableProcessors() * 0.5, 1); } @Config(value = "${base_path}.merge.task.targetRunTimeMillis") From 0e69cd2c2687d287ff4022a99e9de07b12c7715f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 19 Oct 2019 04:09:48 -0700 Subject: [PATCH 33/51] fix initial start delay, lol --- ...rallelMergeCombiningSequenceBenchmark.java | 56 ++++++++++++------- .../ParallelMergeCombiningSequenceTest.java | 47 ++++++++++++---- 2 files changed, 70 insertions(+), 33 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java index 9cb5a471cedf..bc4b3e09b793 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java @@ -59,8 +59,8 @@ @State(Scope.Benchmark) @Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") -@Warmup(iterations = 10) -@Measurement(iterations = 30) +@Warmup(iterations = 5) +@Measurement(iterations = 25) public class ParallelMergeCombiningSequenceBenchmark { private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); @@ -76,6 +76,7 @@ public class ParallelMergeCombiningSequenceBenchmark private static final ExecutorService consumer = Execs.multiThreaded(64, "mock-http-thread"); // note: parameters are broken down to allow easily commenting out lines to mix and match which benchmarks to run + // also note: don't really run this like it is unless you have days to spare @Param({ "8", "16", @@ -125,14 +126,12 @@ public class ParallelMergeCombiningSequenceBenchmark @Param({ "non-blocking", - "initial-block-random-500ms", - "initial-block-random-5000ms", - "slow-sequence-random-100ms-1ms" + "initial-block-random-100-500ms", + "initial-block-random-4000-5000ms", + "slow-sequence-random-10-100ms-1ms" }) private String inputSequenceType; - private List> inputSequences; - private int parallelism; private int targetTaskTimeMillis; private int batchSize; @@ -146,27 +145,30 @@ public void setup() { String[] inputSequenceTypeSplit = inputSequenceType.split("-"); if ("initial".equals(inputSequenceTypeSplit[0])) { - // e.g. "initial-block-random-{startDelay}ms" - final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[3].substring(0, inputSequenceTypeSplit[3].length() - 2)); + // e.g. "initial-block-random-{startDelayStart}-{startDelayEnd}ms" + final int startMillis = Integer.parseInt(inputSequenceTypeSplit[3]); + final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.initialDelaySequence(rowsPerSequence, delayMillis); + ParallelMergeCombiningSequenceTest.initialDelaySequence(rowsPerSequence, delayMillis, delayMillis - startMillis); } else if("slow".equals(inputSequenceTypeSplit[0])) { - // e.g. "slow-sequence-random-{startDelay}ms-{frequencyDelay}ms" - final int startDelayMillis = Integer.parseInt(inputSequenceTypeSplit[3].substring(0,inputSequenceTypeSplit[3].length() - 2)); - final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); + // e.g. "slow-sequence-random-{startDelayStart}-{startDelayEnd}ms-{frequencyDelay}ms" + final int startStartDelayMillis = Integer.parseInt(inputSequenceTypeSplit[3]); + final int startDelayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0,inputSequenceTypeSplit[4].length() - 2)); + final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2)); final int frequency = rowsPerSequence / 10; inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.slowSequence(rowsPerSequence, frequency, startDelayMillis, delayMillis); + ParallelMergeCombiningSequenceTest.slowSequence( + rowsPerSequence, + frequency, + startDelayMillis, + startDelayMillis - startStartDelayMillis, + delayMillis + ); } else { // non-blocking sequence inputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.generateOrderedPairsSequence(rowsPerSequence); } - inputSequences = new ArrayList<>(numSequences); - for (int i = 0; i < numSequences; i++) { - inputSequences.add(inputSequenceFactory.get()); - } - String[] strategySplit = strategy.split("-"); if ("parallelism".equals(strategySplit[0])) { // "parallelism-{parallelism}-{targetTime}ms-{batchSize}-{yieldAfter}" @@ -204,11 +206,20 @@ public void exec(Blackhole blackhole) throws Exception blackhole.consume(futures); } + private List> createInputSequences() + { + List> inputSequences = new ArrayList<>(numSequences); + for (int j = 0; j < numSequences; j++) { + inputSequences.add(inputSequenceFactory.get()); + } + return inputSequences; + } + private Sequence createParallelSequence() { return new ParallelMergeCombiningSequence<>( mergePool, - inputSequences, + createInputSequences(), ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN, false, @@ -224,7 +235,10 @@ private Sequence createParallelSeque private Sequence createCombiningMergeSequence() { return CombiningSequence.create( - new MergeSequence<>(ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, Sequences.simple(inputSequences)), + new MergeSequence<>( + ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, + Sequences.simple(createInputSequences()) + ), ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN ); diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index e7f9b9ad6eda..ec3e89cb2176 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -448,8 +448,7 @@ public void testTimeoutExceptionDueToStalledInput() throws Exception input.add(generateOrderedPairsSequence(someSize)); input.add(generateOrderedPairsSequence(someSize)); input.add(generateOrderedPairsSequence(someSize)); - input.add(slowSequence(someSize, 1, 500, 500)); - + input.add(slowSequence(someSize, 1, 500, 100, 500)); expectedException.expect(RuntimeException.class); expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); expectedException.expectMessage("Sequence iterator timed out waiting for data"); @@ -655,9 +654,14 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence initialDelaySequence(int size, int maxDelayMillis) + public static Sequence initialDelaySequence(int size, int maxDelayMillis, int startDelayRange) { List items = generateOrderedPairs(size); + final long delayMillis = ThreadLocalRandom.current().nextLong( + Math.max(0, maxDelayMillis - startDelayRange), + maxDelayMillis + ); + final long delayUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(delayMillis, TimeUnit.MILLISECONDS); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -676,9 +680,12 @@ public boolean hasNext() @Override public IntPair next() { - if (i == 0) { + final long currentNano = System.nanoTime(); + if (i == 0 && currentNano < delayUntil) { try { - Thread.sleep(ThreadLocalRandom.current().nextInt(1, maxDelayMillis)); + Thread.sleep( + TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS) + ); } catch (InterruptedException ex) { throw new RuntimeException(ex); @@ -698,9 +705,20 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence slowSequence(int size, int frequency, int maxStartDelayMillis, int maxDelayMillis) + public static Sequence slowSequence( + int size, + int frequency, + int maxStartDelayMillis, + int startDelayRange, + int maxDelayMillis + ) { List items = generateOrderedPairs(size); + final long delayMillis = ThreadLocalRandom.current().nextLong( + Math.max(0, maxStartDelayMillis - startDelayRange), + maxStartDelayMillis + ); + final long delayUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(delayMillis, TimeUnit.MILLISECONDS); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -719,15 +737,20 @@ public boolean hasNext() @Override public IntPair next() { - if (i == 0 || (i % frequency == 0 && ThreadLocalRandom.current().nextBoolean())) { - try { + try { + final long currentNano = System.nanoTime(); + if (i == 0 && currentNano < delayUntil) { Thread.sleep( - ThreadLocalRandom.current().nextInt(1, i == 0 ? maxStartDelayMillis : maxDelayMillis) + TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS) + ); + } else if (i > 0 && i % frequency == 0 && ThreadLocalRandom.current().nextBoolean()) { + Thread.sleep( + ThreadLocalRandom.current().nextInt(1, maxDelayMillis) ); } - catch (InterruptedException ex) { - throw new RuntimeException(ex); - } + } + catch (InterruptedException ex) { + throw new RuntimeException(ex); } return items.get(i++); } From c722c164c3cc2f8da38a50ae2df50128c4b91df8 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 22 Oct 2019 01:31:33 -0700 Subject: [PATCH 34/51] parallelism computer now balances partition sizes to partition counts using sqrt of sequence count instead of sequence count by 2 --- .../guava/ParallelMergeCombiningSequence.java | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 7850abb93ac2..5efed27229e1 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -385,21 +385,28 @@ private int computeNumTasks() { final int runningThreadCount = getPool().getRunningThreadCount(); final int submissionCount = getPool().getQueuedSubmissionCount(); - // max is smaller of either parallelism passed into sequence (number of physical cores by default) or the pool - // parallelism + + // max is smaller of either: + // - parallelism passed into sequence (number of physical cores by default) + // - pool parallelism (number of physical cores * 1.5 by default) final int maxParallelism = Math.min(parallelism, getPool().getParallelism()); - // adjust max to be no more than total pool parallelism less the number of running threads + submitted tasks - // minus 1 for the task that is running this calculation since it will be replaced with the parallel tasks + + // we consider 'utilization' to be the number of running threads + submitted tasks that have not yet started + // running, minus 1 for the task that is running this calculation (as it will be replaced with the parallel tasks) final int utilizationEstimate = runningThreadCount + submissionCount - 1; + // 'computed parallelism' is the remaineder of the 'max parallelism' less current 'utilization estimate' - final int computedParallelism = maxParallelism - utilizationEstimate; - // compute total number of layer 1 'parallel' tasks, the final merge task will take the remaining slot - // we divide the sequences by 2 because we need at least 2 sequences per partition for it to make sense to need - // an additional parallel task to compute the merge, so if we have a small number of total sequences this might be - // below + final int computedParallelismForUtilization = maxParallelism - utilizationEstimate; + + // try to balance partition size with partition count so we don't end up with layer 2 'final merge' task that has + // significantly more work to do than the layer 1 'parallel' tasks. + final int computedParallelismForSequences = (int) Math.floor(Math.sqrt(sequences.size())); + + // compute total number of layer 1 'parallel' tasks, for the utilzation parallelism, subtract 1 as the final merge + // task will take the remaining slot final int computedOptimalParallelism = Math.min( - (int) Math.floor((double) sequences.size() / 2.0), - computedParallelism - 1 + computedParallelismForSequences, + computedParallelismForUtilization - 1 ); final int computedNumParallelTasks = Math.max(computedOptimalParallelism, 1); From a9d421308a9beb752762e621d467d40050e182ca Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 22 Oct 2019 03:53:28 -0700 Subject: [PATCH 35/51] fix those important style issues with the benchmarks code --- ...ParallelMergeCombiningSequenceBenchmark.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java index bc4b3e09b793..f1af44a8518f 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java @@ -65,7 +65,7 @@ public class ParallelMergeCombiningSequenceBenchmark { private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); // default merge FJP size - private static final ForkJoinPool mergePool = new ForkJoinPool( + private static final ForkJoinPool MERGE_POOL = new ForkJoinPool( (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.75), ForkJoinPool.defaultForkJoinWorkerThreadFactory, (t, e) -> log.error(e, "Unhandled exception in thread [%s]", t), @@ -73,7 +73,7 @@ public class ParallelMergeCombiningSequenceBenchmark ); // this should be as large as the largest value of concurrentSequenceConsumers - private static final ExecutorService consumer = Execs.multiThreaded(64, "mock-http-thread"); + private static final ExecutorService CONSUMER_POOL = Execs.multiThreaded(64, "mock-http-thread"); // note: parameters are broken down to allow easily commenting out lines to mix and match which benchmarks to run // also note: don't really run this like it is unless you have days to spare @@ -150,10 +150,10 @@ public void setup() final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); inputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.initialDelaySequence(rowsPerSequence, delayMillis, delayMillis - startMillis); - } else if("slow".equals(inputSequenceTypeSplit[0])) { + } else if ("slow".equals(inputSequenceTypeSplit[0])) { // e.g. "slow-sequence-random-{startDelayStart}-{startDelayEnd}ms-{frequencyDelay}ms" final int startStartDelayMillis = Integer.parseInt(inputSequenceTypeSplit[3]); - final int startDelayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0,inputSequenceTypeSplit[4].length() - 2)); + final int startDelayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2)); final int frequency = rowsPerSequence / 10; inputSequenceFactory = () -> @@ -173,7 +173,7 @@ public void setup() if ("parallelism".equals(strategySplit[0])) { // "parallelism-{parallelism}-{targetTime}ms-{batchSize}-{yieldAfter}" parallelism = Integer.parseInt(strategySplit[1]); - targetTaskTimeMillis = Integer.parseInt(strategySplit[2].substring(0,strategySplit[2].length() - 2)); + targetTaskTimeMillis = Integer.parseInt(strategySplit[2].substring(0, strategySplit[2].length() - 2)); batchSize = Integer.parseInt(strategySplit[3]); yieldAfter = Integer.parseInt(strategySplit[4]); outputSequenceFactory = () -> createParallelSequence(); @@ -190,10 +190,11 @@ public void exec(Blackhole blackhole) throws Exception List futures = new ArrayList<>(concurrentSequenceConsumers); for (int i = 0; i < concurrentSequenceConsumers; i++) { futures.add( - consumer.submit(() -> { + CONSUMER_POOL.submit(() -> { try { consumeSequence(blackhole); - } catch (Exception anyException) { + } + catch (Exception anyException) { log.error(anyException, "benchmark failed"); } }) @@ -218,7 +219,7 @@ private List> createInputSe private Sequence createParallelSequence() { return new ParallelMergeCombiningSequence<>( - mergePool, + MERGE_POOL, createInputSequences(), ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN, From d6137b190e1153da6be8edbf04fccdff867e28f2 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 22 Oct 2019 19:25:11 -0700 Subject: [PATCH 36/51] lazy sequence creation for benchmarks --- ...rallelMergeCombiningSequenceBenchmark.java | 12 +- .../ParallelMergeCombiningSequenceTest.java | 143 ++++++++++++++---- 2 files changed, 125 insertions(+), 30 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java index f1af44a8518f..7bdbc3343197 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java @@ -149,7 +149,12 @@ public void setup() final int startMillis = Integer.parseInt(inputSequenceTypeSplit[3]); final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.initialDelaySequence(rowsPerSequence, delayMillis, delayMillis - startMillis); + ParallelMergeCombiningSequenceTest.initialDelaySequence( + rowsPerSequence, + delayMillis, + delayMillis - startMillis, + true + ); } else if ("slow".equals(inputSequenceTypeSplit[0])) { // e.g. "slow-sequence-random-{startDelayStart}-{startDelayEnd}ms-{frequencyDelay}ms" final int startStartDelayMillis = Integer.parseInt(inputSequenceTypeSplit[3]); @@ -162,11 +167,12 @@ public void setup() frequency, startDelayMillis, startDelayMillis - startStartDelayMillis, - delayMillis + delayMillis, + true ); } else { // non-blocking sequence inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.generateOrderedPairsSequence(rowsPerSequence); + ParallelMergeCombiningSequenceTest.generateOrderedPairsSequence(rowsPerSequence, true); } String[] strategySplit = strategy.split("-"); diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index ec3e89cb2176..42333e40d3c4 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -448,7 +448,7 @@ public void testTimeoutExceptionDueToStalledInput() throws Exception input.add(generateOrderedPairsSequence(someSize)); input.add(generateOrderedPairsSequence(someSize)); input.add(generateOrderedPairsSequence(someSize)); - input.add(slowSequence(someSize, 1, 500, 100, 500)); + input.add(slowSequence(someSize, 1, 500, 100, 500, true)); expectedException.expect(RuntimeException.class); expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); expectedException.expectMessage("Sequence iterator timed out waiting for data"); @@ -598,27 +598,75 @@ public IntPair(@Nullable Integer lhs, @Nullable Integer rhs) public static List generateOrderedPairs(int length) { - int counter = 0; - int i = 0; + int rowCounter = 0; + int mergeKey = 0; List generatedSequence = new ArrayList<>(length); - while (counter < length) { - i++; + while (rowCounter < length) { + mergeKey++; if (ThreadLocalRandom.current().nextBoolean()) { - generatedSequence.add(new IntPair(i, ThreadLocalRandom.current().nextInt(1, 100))); - counter++; + generatedSequence.add(makeIntPair(mergeKey)); + rowCounter++; } } return generatedSequence; } - public static Sequence generateOrderedPairsSequence(int length) + public static Sequence generateOrderedPairsSequence(int size) { - return Sequences.simple(generateOrderedPairs(length)); + return generateOrderedPairsSequence(size, false); + } + + public static Sequence generateOrderedPairsSequence(int size, boolean lazyEvaluate) + { + List pairs = lazyEvaluate ? null : generateOrderedPairs(size); + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + int mergeKey = 0; + int rowCounter = 0; + @Override + public boolean hasNext() + { + return rowCounter < size; + } + + @Override + public IntPair next() + { + if (lazyEvaluate) { + IntPair value = null; + while (value == null) { + mergeKey++; + if (ThreadLocalRandom.current().nextBoolean()) { + value = makeIntPair(mergeKey); + rowCounter++; + } + } + return value; + } else { + return pairs.get(rowCounter++); + } + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } + } + ); } public static Sequence explodingSequence(int explodeAfter) { - List items = generateOrderedPairs(explodeAfter + 1); + final int explodeAt = explodeAfter + 1; return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -627,20 +675,29 @@ public Iterator make() { return new Iterator() { - int i = 0; + int mergeKey = 0; + int rowCounter = 0; @Override public boolean hasNext() { - return i < items.size(); + return rowCounter < explodeAt; } @Override public IntPair next() { - if (i == explodeAfter) { + if (rowCounter == explodeAfter) { throw new RuntimeException("exploded"); } - return items.get(i++); + IntPair value = null; + while (value == null) { + mergeKey++; + if (ThreadLocalRandom.current().nextBoolean()) { + value = makeIntPair(mergeKey); + rowCounter++; + } + } + return value; } }; } @@ -654,9 +711,9 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence initialDelaySequence(int size, int maxDelayMillis, int startDelayRange) + public static Sequence initialDelaySequence(int size, int maxDelayMillis, int startDelayRange, boolean lazyEvaluate) { - List items = generateOrderedPairs(size); + List pairs = lazyEvaluate ? null : generateOrderedPairs(size); final long delayMillis = ThreadLocalRandom.current().nextLong( Math.max(0, maxDelayMillis - startDelayRange), maxDelayMillis @@ -670,18 +727,19 @@ public Iterator make() { return new Iterator() { - int i = 0; + int mergeKey = 0; + int rowCounter = 0; @Override public boolean hasNext() { - return i < items.size(); + return rowCounter < size; } @Override public IntPair next() { final long currentNano = System.nanoTime(); - if (i == 0 && currentNano < delayUntil) { + if (rowCounter == 0 && currentNano < delayUntil) { try { Thread.sleep( TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS) @@ -691,7 +749,19 @@ public IntPair next() throw new RuntimeException(ex); } } - return items.get(i++); + if (lazyEvaluate) { + IntPair value = null; + while (value == null) { + mergeKey++; + if (ThreadLocalRandom.current().nextBoolean()) { + value = makeIntPair(mergeKey); + rowCounter++; + } + } + return value; + } else { + return pairs.get(rowCounter++); + } } }; } @@ -710,10 +780,11 @@ public static Sequence slowSequence( int frequency, int maxStartDelayMillis, int startDelayRange, - int maxDelayMillis + int maxDelayMillis, + boolean lazyEvaluate ) { - List items = generateOrderedPairs(size); + List pairs = lazyEvaluate ? null : generateOrderedPairs(size); final long delayMillis = ThreadLocalRandom.current().nextLong( Math.max(0, maxStartDelayMillis - startDelayRange), maxStartDelayMillis @@ -727,11 +798,12 @@ public Iterator make() { return new Iterator() { - int i = 0; + int mergeKey = 0; + int rowCounter = 0; @Override public boolean hasNext() { - return i < items.size(); + return rowCounter < size; } @Override @@ -739,11 +811,11 @@ public IntPair next() { try { final long currentNano = System.nanoTime(); - if (i == 0 && currentNano < delayUntil) { + if (rowCounter == 0 && currentNano < delayUntil) { Thread.sleep( TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS) ); - } else if (i > 0 && i % frequency == 0 && ThreadLocalRandom.current().nextBoolean()) { + } else if (rowCounter > 0 && rowCounter % frequency == 0 && ThreadLocalRandom.current().nextBoolean()) { Thread.sleep( ThreadLocalRandom.current().nextInt(1, maxDelayMillis) ); @@ -752,7 +824,19 @@ public IntPair next() catch (InterruptedException ex) { throw new RuntimeException(ex); } - return items.get(i++); + if (lazyEvaluate) { + IntPair value = null; + while (value == null) { + mergeKey++; + if (ThreadLocalRandom.current().nextBoolean()) { + value = makeIntPair(mergeKey); + rowCounter++; + } + } + return value; + } else { + return pairs.get(rowCounter++); + } } }; } @@ -765,4 +849,9 @@ public void cleanup(Iterator iterFromMake) } ); } + + static IntPair makeIntPair(int mergeKey) + { + return new IntPair(mergeKey, ThreadLocalRandom.current().nextInt(1, 100)); + } } From f5cd469746c9651e33975b1b361eaca7c17f8619 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 22 Oct 2019 19:33:20 -0700 Subject: [PATCH 37/51] more benchmark comments --- .../benchmark/ParallelMergeCombiningSequenceBenchmark.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java index 7bdbc3343197..58924ccfdc79 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java @@ -126,6 +126,10 @@ public class ParallelMergeCombiningSequenceBenchmark @Param({ "non-blocking", + // note: beware when using the blocking sequences for a direct comparison between strategies + // at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable + // between strategies or number of sequences, much less between iterations of the same strategy, compensation in + // the form of running a lot of iterations could potentially make them more directly comparable "initial-block-random-100-500ms", "initial-block-random-4000-5000ms", "slow-sequence-random-10-100ms-1ms" From 4bb3796c73dc891939d7e8eb5babe7f613c43fda Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 23 Oct 2019 01:09:37 -0700 Subject: [PATCH 38/51] stable sequence generation time --- .../ParallelMergeCombiningSequenceTest.java | 61 ++++++------------- 1 file changed, 20 insertions(+), 41 deletions(-) diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index 42333e40d3c4..409e3ef9826b 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -602,11 +602,9 @@ public static List generateOrderedPairs(int length) int mergeKey = 0; List generatedSequence = new ArrayList<>(length); while (rowCounter < length) { - mergeKey++; - if (ThreadLocalRandom.current().nextBoolean()) { - generatedSequence.add(makeIntPair(mergeKey)); - rowCounter++; - } + mergeKey += incrementMergeKeyAmount(); + generatedSequence.add(makeIntPair(mergeKey)); + rowCounter++; } return generatedSequence; } @@ -639,15 +637,9 @@ public boolean hasNext() public IntPair next() { if (lazyEvaluate) { - IntPair value = null; - while (value == null) { - mergeKey++; - if (ThreadLocalRandom.current().nextBoolean()) { - value = makeIntPair(mergeKey); - rowCounter++; - } - } - return value; + rowCounter++; + mergeKey += incrementMergeKeyAmount(); + return makeIntPair(mergeKey); } else { return pairs.get(rowCounter++); } @@ -689,15 +681,9 @@ public IntPair next() if (rowCounter == explodeAfter) { throw new RuntimeException("exploded"); } - IntPair value = null; - while (value == null) { - mergeKey++; - if (ThreadLocalRandom.current().nextBoolean()) { - value = makeIntPair(mergeKey); - rowCounter++; - } - } - return value; + mergeKey += incrementMergeKeyAmount(); + rowCounter++; + return makeIntPair(mergeKey); } }; } @@ -750,15 +736,9 @@ public IntPair next() } } if (lazyEvaluate) { - IntPair value = null; - while (value == null) { - mergeKey++; - if (ThreadLocalRandom.current().nextBoolean()) { - value = makeIntPair(mergeKey); - rowCounter++; - } - } - return value; + rowCounter++; + mergeKey += incrementMergeKeyAmount(); + return makeIntPair(mergeKey); } else { return pairs.get(rowCounter++); } @@ -825,15 +805,9 @@ public IntPair next() throw new RuntimeException(ex); } if (lazyEvaluate) { - IntPair value = null; - while (value == null) { - mergeKey++; - if (ThreadLocalRandom.current().nextBoolean()) { - value = makeIntPair(mergeKey); - rowCounter++; - } - } - return value; + rowCounter++; + mergeKey += incrementMergeKeyAmount(); + return makeIntPair(mergeKey); } else { return pairs.get(rowCounter++); } @@ -850,6 +824,11 @@ public void cleanup(Iterator iterFromMake) ); } + static int incrementMergeKeyAmount() + { + return ThreadLocalRandom.current().nextInt(1, 3); + } + static IntPair makeIntPair(int mergeKey) { return new IntPair(mergeKey, ThreadLocalRandom.current().nextInt(1, 100)); From 0b26c50c7573179df67be051c3742fc4557ffb5b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 23 Oct 2019 13:52:54 -0700 Subject: [PATCH 39/51] update defaults to use 100ms target time, 4096 batch size, 16384 initial yield, also update user docs --- .../guava/ParallelMergeCombiningSequence.java | 8 +++++--- docs/configuration/index.md | 13 +++++++------ 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 5efed27229e1..41a82c219a1c 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -57,9 +57,11 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { private static final Logger LOG = new Logger(ParallelMergeCombiningSequence.class); - public static final int DEFAULT_TASK_TARGET_RUN_TIME_MILLIS = 10; - public static final int DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS = 1024; - public static final int DEFAULT_TASK_SMALL_BATCH_NUM_ROWS = 128; + // these values were chosen carefully via feedback from benchmarks, + // see PR https://github.com/apache/incubator-druid/pull/8578 for details + public static final int DEFAULT_TASK_TARGET_RUN_TIME_MILLIS = 100; + public static final int DEFAULT_TASK_INITIAL_YIELD_NUM_ROWS = 16384; + public static final int DEFAULT_TASK_SMALL_BATCH_NUM_ROWS = 4096; private final ForkJoinPool workerPool; private final List> baseSequences; diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 8f7826c46abe..649b11810f70 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1435,12 +1435,13 @@ The broker uses processing configs for nested groupBy queries. And, if you use g |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| -|`druid.processing.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`| -|`druid.processing.numMergePoolThreads`|Size of ForkJoinPool|`druid.processing.numThreads * 1.5` (rounded up)| -|`druid.processing.mergePoolAwaitShutdownMillis`|Time to wait for merge ForkJoinPool tasks to complete before ungracefully stopping on process shutdown in milliseconds.|`60_000`| -|`druid.processing.mergePoolDefaultMaxParallelism`|Default per query maximum number of parallel merge tasks.|`unlimited`| -|`druid.processing.mergePoolTaskInitialYieldRows`|Number of rows to yield per ForkJoinPool merge task, before forking off a new task to continue merging sequences.|`1024`| -|`druid.processing.mergePoolSmallBatchRows`|Size of result batches to operate on in ForkJoinPool merge tasks.|`128`| +|`druid.processing.merge.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`| +|`druid.processing.merge.pool.parallelism`|Size of ForkJoinPool. Note that the default configuration assumes that the value returned by `Runtime.getRuntime().availableProcessors()` represents 2 hyper-threads per physical core, and multiplies this value by `0.75` in attempt to size `1.5` times the number of _physical_ cores.|`Runtime.getRuntime().availableProcessors() * 0.75` (rounded up)| +|`druid.processing.merge.pool.defaultMaxQueryParallelism`|Default maximum number of parallel merge tasks per query. Note that the default configuration assumes that the value returned by `Runtime.getRuntime().availableProcessors()` represents 2 hyper-threads per physical core, and multiplies this value by `0.5` in attempt to size to the number of _physical_ cores.|`Runtime.getRuntime().availableProcessors() * 0.5` (rounded up)| +|`druid.processing.merge.pool.awaitShutdownMillis`|Time to wait for merge ForkJoinPool tasks to complete before ungracefully stopping on process shutdown in milliseconds.|`60_000`| +|`druid.processing.merge.task.targetRunTimeMillis`|Ideal run-time of each ForkJoinPool merge task, before forking off a new task to continue merging sequences.|`100`| +|`druid.processing.merge.task.initialYieldNumRows`|Number of rows to yield per ForkJoinPool merge task, before forking off a new task to continue merging sequences.|`16384`| +|`druid.processing.merge.task.smallBatchNumRows`|Size of result batches to operate on in ForkJoinPool merge tasks.|`4096`| The amount of direct memory needed by Druid is at least `druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can From 9aa3d0ba1e65752156f4c648ba79f4149e4750b3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 28 Oct 2019 15:51:05 -0700 Subject: [PATCH 40/51] add jmh thread based benchmarks, cleanup some stuff --- ...allelMergeCombiningSequenceBenchmark.java} | 178 ++++------ .../MergeSequenceBenchmark.java | 2 +- ...rallelMergeCombiningSequenceBenchmark.java | 116 +++++++ ...eCombiningSequenceJmhThreadsBenchmark.java | 143 ++++++++ .../ParallelMergeCombiningSequenceTest.java | 315 ++++++++---------- 5 files changed, 461 insertions(+), 293 deletions(-) rename benchmarks/src/main/java/org/apache/druid/benchmark/{ParallelMergeCombiningSequenceBenchmark.java => sequences/BaseParallelMergeCombiningSequenceBenchmark.java} (53%) rename benchmarks/src/main/java/org/apache/druid/benchmark/{ => sequences}/MergeSequenceBenchmark.java (98%) create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java similarity index 53% rename from benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java rename to benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index 58924ccfdc79..3493fa02a840 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark; +package org.apache.druid.benchmark.sequences; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.java.util.common.concurrent.Execs; @@ -29,53 +29,33 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; -import org.openjdk.jmh.results.format.ResultFormatType; -import org.openjdk.jmh.runner.Runner; -import org.openjdk.jmh.runner.RunnerException; -import org.openjdk.jmh.runner.options.Options; -import org.openjdk.jmh.runner.options.OptionsBuilder; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.function.Supplier; - @State(Scope.Benchmark) -@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") -@Warmup(iterations = 5) -@Measurement(iterations = 25) -public class ParallelMergeCombiningSequenceBenchmark +public class BaseParallelMergeCombiningSequenceBenchmark { private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); // default merge FJP size - private static final ForkJoinPool MERGE_POOL = new ForkJoinPool( + static final ForkJoinPool MERGE_POOL = new ForkJoinPool( (int) Math.ceil(Runtime.getRuntime().availableProcessors() * 0.75), ForkJoinPool.defaultForkJoinWorkerThreadFactory, (t, e) -> log.error(e, "Unhandled exception in thread [%s]", t), true ); - // this should be as large as the largest value of concurrentSequenceConsumers - private static final ExecutorService CONSUMER_POOL = Execs.multiThreaded(64, "mock-http-thread"); - - // note: parameters are broken down to allow easily commenting out lines to mix and match which benchmarks to run + // note: parameters are broken down one per line to allow easily commenting out lines to mix and match which + // benchmarks to run // also note: don't really run this like it is unless you have days to spare @Param({ "8", @@ -83,21 +63,14 @@ public class ParallelMergeCombiningSequenceBenchmark "32", "64" }) - private int numSequences; - - @Param({"75000"}) - private int rowsPerSequence; + int numSequences; @Param({ - "1", - "2", - "4", - "8", - "16", - "32", - "64" + "1000", + "75000", + "10000000" }) - private int concurrentSequenceConsumers; + int rowsPerSequence; @Param({ "combiningMergeSequence-same-thread", @@ -105,36 +78,29 @@ public class ParallelMergeCombiningSequenceBenchmark "parallelism-4-10ms-256-1024", "parallelism-8-10ms-256-1024", "parallelism-16-10ms-256-1024", - "parallelism-1-100ms-512-4096", - "parallelism-4-100ms-512-4096", - "parallelism-8-100ms-512-4096", - "parallelism-16-100ms-512-4096", "parallelism-1-100ms-1024-4096", "parallelism-4-100ms-1024-4096", "parallelism-8-100ms-1024-4096", "parallelism-16-100ms-1024-4096", - "parallelism-1-100ms-1024-16384", - "parallelism-4-100ms-1024-16384", - "parallelism-8-100ms-1024-16384", - "parallelism-16-100ms-1024-16384", "parallelism-1-100ms-4096-16384", "parallelism-4-100ms-4096-16384", "parallelism-8-100ms-4096-16384", "parallelism-16-100ms-4096-16384" }) - private String strategy; + String strategy; + @Param({ - "non-blocking", + "non-blocking-sequence", // note: beware when using the blocking sequences for a direct comparison between strategies // at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable // between strategies or number of sequences, much less between iterations of the same strategy, compensation in // the form of running a lot of iterations could potentially make them more directly comparable - "initial-block-random-100-500ms", - "initial-block-random-4000-5000ms", - "slow-sequence-random-10-100ms-1ms" + "initially-blocking-sequence-100-500ms", + "initially-blocking-sequence-4000-5000ms", + "blocking-sequence-10-100ms-10-1ms" }) - private String inputSequenceType; + String inputSequenceType; private int parallelism; private int targetTaskTimeMillis; @@ -148,35 +114,44 @@ public class ParallelMergeCombiningSequenceBenchmark public void setup() { String[] inputSequenceTypeSplit = inputSequenceType.split("-"); - if ("initial".equals(inputSequenceTypeSplit[0])) { - // e.g. "initial-block-random-{startDelayStart}-{startDelayEnd}ms" - final int startMillis = Integer.parseInt(inputSequenceTypeSplit[3]); - final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); + if ("initially".equals(inputSequenceTypeSplit[0])) { + // e.g. "initially-blocking-sequence-{startDelayStart}-{startDelayEnd}ms" + final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[3]); + final int startDelayEndMillis = Integer.parseInt( + inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2) + ); inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.initialDelaySequence( + ParallelMergeCombiningSequenceTest.blockingSequence( rowsPerSequence, - delayMillis, - delayMillis - startMillis, + startDelayStartMillis, + startDelayEndMillis, + -1, + 0, true ); - } else if ("slow".equals(inputSequenceTypeSplit[0])) { - // e.g. "slow-sequence-random-{startDelayStart}-{startDelayEnd}ms-{frequencyDelay}ms" - final int startStartDelayMillis = Integer.parseInt(inputSequenceTypeSplit[3]); - final int startDelayMillis = Integer.parseInt(inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2)); - final int delayMillis = Integer.parseInt(inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2)); - final int frequency = rowsPerSequence / 10; + } else if ("blocking".equals(inputSequenceTypeSplit[0])) { + // e.g. "blocking-sequence-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms" + final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[2]); + final int startDelayEndMillis = Integer.parseInt( + inputSequenceTypeSplit[3].substring(0, inputSequenceTypeSplit[3].length() - 2) + ); + final int numberOfTimesToBlock = Integer.parseInt(inputSequenceTypeSplit[4]); + final int maxIterationDelayMillis = Integer.parseInt( + inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2) + ); + final int frequency = rowsPerSequence / numberOfTimesToBlock; inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.slowSequence( + ParallelMergeCombiningSequenceTest.blockingSequence( rowsPerSequence, + startDelayStartMillis, + startDelayEndMillis, frequency, - startDelayMillis, - startDelayMillis - startStartDelayMillis, - delayMillis, + maxIterationDelayMillis, true ); } else { // non-blocking sequence inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.generateOrderedPairsSequence(rowsPerSequence, true); + ParallelMergeCombiningSequenceTest.nonBlockingSequence(rowsPerSequence, true); } String[] strategySplit = strategy.split("-"); @@ -192,32 +167,8 @@ public void setup() } } - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void exec(Blackhole blackhole) throws Exception - { - List futures = new ArrayList<>(concurrentSequenceConsumers); - for (int i = 0; i < concurrentSequenceConsumers; i++) { - futures.add( - CONSUMER_POOL.submit(() -> { - try { - consumeSequence(blackhole); - } - catch (Exception anyException) { - log.error(anyException, "benchmark failed"); - } - }) - ); - } - - for (int i = 0; i < concurrentSequenceConsumers; i++) { - blackhole.consume(futures.get(i).get()); - } - blackhole.consume(futures); - } - private List> createInputSequences() + List> createInputSequences() { List> inputSequences = new ArrayList<>(numSequences); for (int j = 0; j < numSequences; j++) { @@ -226,7 +177,7 @@ private List> createInputSe return inputSequences; } - private Sequence createParallelSequence() + Sequence createParallelSequence() { return new ParallelMergeCombiningSequence<>( MERGE_POOL, @@ -243,7 +194,7 @@ private Sequence createParallelSeque ); } - private Sequence createCombiningMergeSequence() + Sequence createCombiningMergeSequence() { return CombiningSequence.create( new MergeSequence<>( @@ -255,28 +206,21 @@ private Sequence createCombiningMerg ); } - private void consumeSequence(Blackhole blackhole) + void consumeSequence(Blackhole blackhole) { - Yielder yielder = Yielders.each(outputSequenceFactory.get()); - - ParallelMergeCombiningSequenceTest.IntPair prev; - while (!yielder.isDone()) { - prev = yielder.get(); - blackhole.consume(prev); - yielder = yielder.next(yielder.get()); + try { + Yielder yielder = Yielders.each(outputSequenceFactory.get()); + + ParallelMergeCombiningSequenceTest.IntPair prev; + while (!yielder.isDone()) { + prev = yielder.get(); + blackhole.consume(prev); + yielder = yielder.next(yielder.get()); + } + } + catch (Exception anyException) { + log.error(anyException, "benchmark failed"); + throw new RuntimeException(anyException); } - } - - public static void main(String[] args) throws RunnerException - { - Options opt = new OptionsBuilder() - .include(ParallelMergeCombiningSequenceBenchmark.class.getSimpleName()) - .forks(1) - .syncIterations(true) - .resultFormat(ResultFormatType.CSV) - .result("parallel-merge-combining-sequence.csv") - .build(); - - new Runner(opt).run(); } } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/MergeSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/MergeSequenceBenchmark.java similarity index 98% rename from benchmarks/src/main/java/org/apache/druid/benchmark/MergeSequenceBenchmark.java rename to benchmarks/src/main/java/org/apache/druid/benchmark/sequences/MergeSequenceBenchmark.java index 150bddcb8ef9..995163077e50 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/MergeSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/MergeSequenceBenchmark.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark; +package org.apache.druid.benchmark.sequences; import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java new file mode 100644 index 000000000000..38acf2a9bf69 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java @@ -0,0 +1,116 @@ +/* + * 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.benchmark.sequences; + +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + + +@State(Scope.Benchmark) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +@Warmup(iterations = 5) +@Measurement(iterations = 10) +public class ParallelMergeCombiningSequenceBenchmark extends BaseParallelMergeCombiningSequenceBenchmark +{ + private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); + // this should be as large as the largest value of concurrentSequenceConsumers + private static final ExecutorService CONSUMER_POOL = Execs.multiThreaded(64, "mock-http-thread"); + + @Param({ + "1", + "2", + "4", + "8", + "16", + "32", + "64" + }) + private int concurrentSequenceConsumers; + + @Param({ + "0", + "10", + "100", + "500", + "1000" + }) + private int concurrentConsumerDelayMillis; + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void exec(Blackhole blackhole) throws Exception + { + List futures = createConsumers(blackhole, concurrentSequenceConsumers, concurrentConsumerDelayMillis); + + for (int i = 0; i < concurrentSequenceConsumers; i++) { + blackhole.consume(futures.get(i).get()); + } + blackhole.consume(futures); + } + + private List createConsumers(Blackhole blackhole, int consumers, int delayMillis) throws Exception + { + List futures = new ArrayList<>(consumers); + for (int i = 0; i < consumers; i++) { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + futures.add(CONSUMER_POOL.submit(() -> consumeSequence(blackhole))); + } + + return futures; + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ParallelMergeCombiningSequenceBenchmark.class.getSimpleName()) + .forks(1) + .syncIterations(true) + .resultFormat(ResultFormatType.CSV) + .result("parallel-merge-combining-sequence.csv") + .build(); + + new Runner(opt).run(); + } +} diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java new file mode 100644 index 000000000000..0cfb92d218b0 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java @@ -0,0 +1,143 @@ +/* + * 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.benchmark.sequences; + +import org.apache.druid.java.util.common.logger.Logger; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.util.concurrent.TimeUnit; + +/** + * This benchmark measures a slightly different thing than + * {@link ParallelMergeCombiningSequenceBenchmark}. While this benchmark measures the + * average time each thread takes to complete per iteration, the other measures the average time for _all_ threads to + * complete defined by {@link ParallelMergeCombiningSequenceBenchmark#concurrentSequenceConsumers} using an + * {@link java.util.concurrent.ExecutorService} thread pool. Additionally, the + * {@link ParallelMergeCombiningSequenceBenchmark} benchmark is also able to introduce a delay between the concurrent + * processing threads to simulate other more realistic patterns than a simulataneous burst of concurrent threads. + * + * Yes, this benchmark is sort of lame, unfortunately I could not figure out how to parameterize @Threads. + */ +@State(Scope.Benchmark) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +@Warmup(iterations = 5) +@Measurement(iterations = 10) +public class ParallelMergeCombiningSequenceJmhThreadsBenchmark extends BaseParallelMergeCombiningSequenceBenchmark +{ + @Threads(1) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execOne(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(2) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execTwo(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(4) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execFour(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(8) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execEight(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(16) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execSixteen(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(24) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execTwentyFour(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(32) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execThirtyTwo(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + @Threads(64) + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void execSixtyFour(Blackhole blackhole) + { + consumeSequence(blackhole); + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ParallelMergeCombiningSequenceJmhThreadsBenchmark.class.getSimpleName()) + .forks(1) + .syncIterations(true) + .resultFormat(ResultFormatType.CSV) + .result("parallel-merge-combining-sequence-threads.csv") + .build(); + + new Runner(opt).run(); + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index 409e3ef9826b..b7d443e32018 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -31,11 +31,11 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Random; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ThreadLocalRandom; @@ -85,7 +85,7 @@ public void teardown() @Test public void testOrderedResultBatchFromSequence() throws IOException { - Sequence rawSequence = generateOrderedPairsSequence(5000); + Sequence rawSequence = nonBlockingSequence(5000); ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), @@ -111,7 +111,7 @@ public void testOrderedResultBatchFromSequenceBackToYielderOnSequence() throws I { final int batchSize = 128; final int sequenceSize = 5_000; - Sequence rawSequence = generateOrderedPairsSequence(sequenceSize); + Sequence rawSequence = nonBlockingSequence(sequenceSize); ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), @@ -178,7 +178,7 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE { final int batchSize = 128; final int sequenceSize = 5_000; - Sequence rawSequence = generateOrderedPairsSequence(sequenceSize); + Sequence rawSequence = nonBlockingSequence(sequenceSize); ParallelMergeCombiningSequence.YielderBatchedResultsCursor cursor = new ParallelMergeCombiningSequence.YielderBatchedResultsCursor<>( new ParallelMergeCombiningSequence.SequenceBatcher<>(rawSequence, 128), @@ -269,7 +269,7 @@ public void testEmptiesAndNonEmpty() throws Exception // below min threshold, so will merge serially List> input = new ArrayList<>(); input.add(Sequences.empty()); - input.add(generateOrderedPairsSequence(5)); + input.add(nonBlockingSequence(5)); assertResult(input); input.clear(); @@ -280,7 +280,7 @@ public void testEmptiesAndNonEmpty() throws Exception input.add(Sequences.empty()); input.add(Sequences.empty()); input.add(Sequences.empty()); - input.add(generateOrderedPairsSequence(5)); + input.add(nonBlockingSequence(5)); assertResult(input); } @@ -289,19 +289,19 @@ public void testAllInSingleBatch() throws Exception { // below min threshold, so will merge serially List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(6)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(6)); assertResult(input, 10, 20); input.clear(); // above min sequence count threshold, so will merge in parallel (if enough cores) - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(6)); - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(8)); - input.add(generateOrderedPairsSequence(4)); - input.add(generateOrderedPairsSequence(6)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(6)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(8)); + input.add(nonBlockingSequence(4)); + input.add(nonBlockingSequence(6)); assertResult(input, 10, 20); } @@ -310,19 +310,19 @@ public void testAllInSingleYield() throws Exception { // below min threshold, so will merge serially List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(6)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(6)); assertResult(input, 4, 20); input.clear(); // above min sequence count threshold, so will merge in parallel (if enough cores) - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(6)); - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(8)); - input.add(generateOrderedPairsSequence(4)); - input.add(generateOrderedPairsSequence(6)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(6)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(8)); + input.add(nonBlockingSequence(4)); + input.add(nonBlockingSequence(6)); assertResult(input, 4, 20); } @@ -332,16 +332,16 @@ public void testMultiBatchMultiYield() throws Exception { // below min threshold, so will merge serially List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(15)); - input.add(generateOrderedPairsSequence(26)); + input.add(nonBlockingSequence(15)); + input.add(nonBlockingSequence(26)); assertResult(input, 5, 10); // above min sequence count threshold, so will merge in parallel (if enough cores) - input.add(generateOrderedPairsSequence(15)); - input.add(generateOrderedPairsSequence(33)); - input.add(generateOrderedPairsSequence(17)); - input.add(generateOrderedPairsSequence(14)); + input.add(nonBlockingSequence(15)); + input.add(nonBlockingSequence(33)); + input.add(nonBlockingSequence(17)); + input.add(nonBlockingSequence(14)); assertResult(input, 5, 10); } @@ -351,16 +351,16 @@ public void testMixedSingleAndMultiYield() throws Exception { // below min threshold, so will merge serially List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(60)); - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(8)); + input.add(nonBlockingSequence(60)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(8)); assertResult(input, 5, 10); // above min sequence count threshold, so will merge in parallel (if enough cores) - input.add(generateOrderedPairsSequence(1)); - input.add(generateOrderedPairsSequence(8)); - input.add(generateOrderedPairsSequence(32)); + input.add(nonBlockingSequence(1)); + input.add(nonBlockingSequence(8)); + input.add(nonBlockingSequence(32)); assertResult(input, 5, 10); } @@ -370,15 +370,15 @@ public void testLongerSequencesJustForFun() throws Exception { List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(10_000)); - input.add(generateOrderedPairsSequence(9_001)); + input.add(nonBlockingSequence(10_000)); + input.add(nonBlockingSequence(9_001)); assertResult(input, 128, 1024); - input.add(generateOrderedPairsSequence(7_777)); - input.add(generateOrderedPairsSequence(8_500)); - input.add(generateOrderedPairsSequence(5_000)); - input.add(generateOrderedPairsSequence(8_888)); + input.add(nonBlockingSequence(7_777)); + input.add(nonBlockingSequence(8_500)); + input.add(nonBlockingSequence(5_000)); + input.add(nonBlockingSequence(8_888)); assertResult(input, 128, 1024); } @@ -389,7 +389,7 @@ public void testExceptionOnInputSequenceRead() throws Exception List> input = new ArrayList<>(); input.add(explodingSequence(15)); - input.add(generateOrderedPairsSequence(25)); + input.add(nonBlockingSequence(25)); expectedException.expect(RuntimeException.class); @@ -398,10 +398,10 @@ public void testExceptionOnInputSequenceRead() throws Exception ); assertException(input); - input.add(generateOrderedPairsSequence(5)); - input.add(generateOrderedPairsSequence(25)); + input.add(nonBlockingSequence(5)); + input.add(nonBlockingSequence(25)); input.add(explodingSequence(11)); - input.add(generateOrderedPairsSequence(12)); + input.add(nonBlockingSequence(12)); assertException(input); } @@ -411,9 +411,9 @@ public void testExceptionFirstResultFromSequence() throws Exception { List> input = new ArrayList<>(); input.add(explodingSequence(0)); - input.add(generateOrderedPairsSequence(2)); - input.add(generateOrderedPairsSequence(2)); - input.add(generateOrderedPairsSequence(2)); + input.add(nonBlockingSequence(2)); + input.add(nonBlockingSequence(2)); + input.add(nonBlockingSequence(2)); expectedException.expect(RuntimeException.class); expectedException.expectMessage( @@ -429,9 +429,9 @@ public void testExceptionFirstResultFromMultipleSequence() throws Exception input.add(explodingSequence(0)); input.add(explodingSequence(0)); input.add(explodingSequence(0)); - input.add(generateOrderedPairsSequence(2)); - input.add(generateOrderedPairsSequence(2)); - input.add(generateOrderedPairsSequence(2)); + input.add(nonBlockingSequence(2)); + input.add(nonBlockingSequence(2)); + input.add(nonBlockingSequence(2)); expectedException.expect(RuntimeException.class); expectedException.expectMessage( @@ -445,10 +445,10 @@ public void testTimeoutExceptionDueToStalledInput() throws Exception { final int someSize = 2048; List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(someSize)); - input.add(generateOrderedPairsSequence(someSize)); - input.add(generateOrderedPairsSequence(someSize)); - input.add(slowSequence(someSize, 1, 500, 100, 500, true)); + input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize)); + input.add(blockingSequence(someSize, 400, 500,1, 500, true)); expectedException.expect(RuntimeException.class); expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); expectedException.expectMessage("Sequence iterator timed out waiting for data"); @@ -467,10 +467,10 @@ public void testTimeoutExceptionDueToStalledReader() throws Exception { final int someSize = 2048; List> input = new ArrayList<>(); - input.add(generateOrderedPairsSequence(someSize)); - input.add(generateOrderedPairsSequence(someSize)); - input.add(generateOrderedPairsSequence(someSize)); - input.add(generateOrderedPairsSequence(someSize)); + input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize)); expectedException.expect(RuntimeException.class); expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); @@ -590,33 +590,22 @@ private void assertException( public static class IntPair extends Pair { - public IntPair(@Nullable Integer lhs, @Nullable Integer rhs) + private IntPair(Integer lhs, Integer rhs) { super(lhs, rhs); } } - public static List generateOrderedPairs(int length) + /** + * Generate an ordered, random valued, non-blocking sequence of {@link IntPair}, optionally lazy generated with + * the implication that every time a sequence is accumulated or yielded it produces different results, + * which sort of breaks the {@link Sequence} contract, and makes this method useless for tests in lazy mode, + * however it is useful for benchmarking, where having a sequence without having to materialize the entire thing + * up front on heap with a {@link List} backing is preferable. + */ + public static Sequence nonBlockingSequence(int size, boolean lazyGenerate) { - int rowCounter = 0; - int mergeKey = 0; - List generatedSequence = new ArrayList<>(length); - while (rowCounter < length) { - mergeKey += incrementMergeKeyAmount(); - generatedSequence.add(makeIntPair(mergeKey)); - rowCounter++; - } - return generatedSequence; - } - - public static Sequence generateOrderedPairsSequence(int size) - { - return generateOrderedPairsSequence(size, false); - } - - public static Sequence generateOrderedPairsSequence(int size, boolean lazyEvaluate) - { - List pairs = lazyEvaluate ? null : generateOrderedPairs(size); + List pairs = lazyGenerate ? null : generateOrderedPairs(size); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -636,7 +625,7 @@ public boolean hasNext() @Override public IntPair next() { - if (lazyEvaluate) { + if (lazyGenerate) { rowCounter++; mergeKey += incrementMergeKeyAmount(); return makeIntPair(mergeKey); @@ -656,55 +645,28 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence explodingSequence(int explodeAfter) - { - final int explodeAt = explodeAfter + 1; - return new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return new Iterator() - { - int mergeKey = 0; - int rowCounter = 0; - @Override - public boolean hasNext() - { - return rowCounter < explodeAt; - } - - @Override - public IntPair next() - { - if (rowCounter == explodeAfter) { - throw new RuntimeException("exploded"); - } - mergeKey += incrementMergeKeyAmount(); - rowCounter++; - return makeIntPair(mergeKey); - } - }; - } - - @Override - public void cleanup(Iterator iterFromMake) - { - // nothing to cleanup - } - } - ); - } - - public static Sequence initialDelaySequence(int size, int maxDelayMillis, int startDelayRange, boolean lazyEvaluate) + /** + * Generate an ordered, random valued, blocking sequence of {@link IntPair}, optionally lazy generated. See + * {@link ParallelMergeCombiningSequenceTest#nonBlockingSequence(int)} for the implications of lazy generating a + * sequence, to summarize each time the sequence is accumulated or yielded it produces different results. + * + * This sequence simulates blocking using {@link Thread#sleep(long)}, with an initial millisecond delay range defined + * by {@param startDelayStartMillis} and {@param startDelayEndMillis} that defines how long to block before the first + * sequence value will be produced, and {@param maxIterationDelayMillis} that defines how long to block every + * {@param iterationDelayFrequency} rows. + */ + public static Sequence blockingSequence( + int size, + int startDelayStartMillis, + int startDelayEndMillis, + int iterationDelayFrequency, + int maxIterationDelayMillis, + boolean lazyGenerate + ) { - List pairs = lazyEvaluate ? null : generateOrderedPairs(size); - final long delayMillis = ThreadLocalRandom.current().nextLong( - Math.max(0, maxDelayMillis - startDelayRange), - maxDelayMillis - ); - final long delayUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(delayMillis, TimeUnit.MILLISECONDS); + final List pairs = lazyGenerate ? null : generateOrderedPairs(size); + final long startDelayMillis = ThreadLocalRandom.current().nextLong(startDelayStartMillis, startDelayEndMillis); + final long delayUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(startDelayMillis, TimeUnit.MILLISECONDS); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -724,18 +686,25 @@ public boolean hasNext() @Override public IntPair next() { - final long currentNano = System.nanoTime(); - if (rowCounter == 0 && currentNano < delayUntil) { - try { - Thread.sleep( - TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS) + try { + final long currentNano = System.nanoTime(); + if (rowCounter == 0 && currentNano < delayUntil) { + final long sleepMillis = Math.max( + TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS), + 1 ); - } - catch (InterruptedException ex) { - throw new RuntimeException(ex); + Thread.sleep(sleepMillis); + } else if (maxIterationDelayMillis > 0 + && rowCounter % iterationDelayFrequency == 0 + && ThreadLocalRandom.current().nextBoolean()) { + final int delayMillis = Math.max(ThreadLocalRandom.current().nextInt(maxIterationDelayMillis), 1); + Thread.sleep(delayMillis); } } - if (lazyEvaluate) { + catch (InterruptedException ex) { + throw new RuntimeException(ex); + } + if (lazyGenerate) { rowCounter++; mergeKey += incrementMergeKeyAmount(); return makeIntPair(mergeKey); @@ -755,21 +724,20 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence slowSequence( - int size, - int frequency, - int maxStartDelayMillis, - int startDelayRange, - int maxDelayMillis, - boolean lazyEvaluate - ) + /** + * Genenerate non-blocking sequence for tests, non-lazy so the sequence produces consistent results + */ + private static Sequence nonBlockingSequence(int size) { - List pairs = lazyEvaluate ? null : generateOrderedPairs(size); - final long delayMillis = ThreadLocalRandom.current().nextLong( - Math.max(0, maxStartDelayMillis - startDelayRange), - maxStartDelayMillis - ); - final long delayUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(delayMillis, TimeUnit.MILLISECONDS); + return nonBlockingSequence(size, false); + } + + /** + * Genenerate a sequence that explodes after {@param explodeAfter} rows + */ + private static Sequence explodingSequence(int explodeAfter) + { + final int explodeAt = explodeAfter + 1; return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -783,34 +751,18 @@ public Iterator make() @Override public boolean hasNext() { - return rowCounter < size; + return rowCounter < explodeAt; } @Override public IntPair next() { - try { - final long currentNano = System.nanoTime(); - if (rowCounter == 0 && currentNano < delayUntil) { - Thread.sleep( - TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS) - ); - } else if (rowCounter > 0 && rowCounter % frequency == 0 && ThreadLocalRandom.current().nextBoolean()) { - Thread.sleep( - ThreadLocalRandom.current().nextInt(1, maxDelayMillis) - ); - } - } - catch (InterruptedException ex) { - throw new RuntimeException(ex); - } - if (lazyEvaluate) { - rowCounter++; - mergeKey += incrementMergeKeyAmount(); - return makeIntPair(mergeKey); - } else { - return pairs.get(rowCounter++); + if (rowCounter == explodeAfter) { + throw new RuntimeException("exploded"); } + mergeKey += incrementMergeKeyAmount(); + rowCounter++; + return makeIntPair(mergeKey); } }; } @@ -824,12 +776,25 @@ public void cleanup(Iterator iterFromMake) ); } - static int incrementMergeKeyAmount() + private static List generateOrderedPairs(int length) + { + int rowCounter = 0; + int mergeKey = 0; + List generatedSequence = new ArrayList<>(length); + while (rowCounter < length) { + mergeKey += incrementMergeKeyAmount(); + generatedSequence.add(makeIntPair(mergeKey)); + rowCounter++; + } + return generatedSequence; + } + + private static int incrementMergeKeyAmount() { return ThreadLocalRandom.current().nextInt(1, 3); } - static IntPair makeIntPair(int mergeKey) + private static IntPair makeIntPair(int mergeKey) { return new IntPair(mergeKey, ThreadLocalRandom.current().nextInt(1, 100)); } From 9fd16c2a9c2fe30ca7f0608ed80143472b31ba9b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 28 Oct 2019 16:49:45 -0700 Subject: [PATCH 41/51] oops --- .../util/common/guava/ParallelMergeCombiningSequenceTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index b7d443e32018..f5691ad554a9 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -35,7 +35,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Random; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ThreadLocalRandom; From 7531859f0c84fd498b333250e06c58e45968562d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 28 Oct 2019 18:19:30 -0700 Subject: [PATCH 42/51] style --- .../BaseParallelMergeCombiningSequenceBenchmark.java | 2 -- .../ParallelMergeCombiningSequenceJmhThreadsBenchmark.java | 1 - .../util/common/guava/ParallelMergeCombiningSequenceTest.java | 4 ++-- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index 3493fa02a840..08a9ce77b8c9 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -20,7 +20,6 @@ package org.apache.druid.benchmark.sequences; import org.apache.druid.common.guava.CombiningSequence; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.MergeSequence; import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence; import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequenceTest; @@ -38,7 +37,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ForkJoinPool; import java.util.function.Supplier; diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java index 0cfb92d218b0..fd2033aa001a 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java @@ -19,7 +19,6 @@ package org.apache.druid.benchmark.sequences; -import org.apache.druid.java.util.common.logger.Logger; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index f5691ad554a9..156447505c4b 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -447,7 +447,7 @@ public void testTimeoutExceptionDueToStalledInput() throws Exception input.add(nonBlockingSequence(someSize)); input.add(nonBlockingSequence(someSize)); input.add(nonBlockingSequence(someSize)); - input.add(blockingSequence(someSize, 400, 500,1, 500, true)); + input.add(blockingSequence(someSize, 400, 500, 1, 500, true)); expectedException.expect(RuntimeException.class); expectedException.expectCause(Matchers.instanceOf(TimeoutException.class)); expectedException.expectMessage("Sequence iterator timed out waiting for data"); @@ -689,7 +689,7 @@ public IntPair next() final long currentNano = System.nanoTime(); if (rowCounter == 0 && currentNano < delayUntil) { final long sleepMillis = Math.max( - TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS), + TimeUnit.MILLISECONDS.convert(delayUntil - currentNano, TimeUnit.NANOSECONDS), 1 ); Thread.sleep(sleepMillis); From 8697c2a23da216fbe4026d987245b5225c9b7bed Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 30 Oct 2019 12:42:30 -0700 Subject: [PATCH 43/51] add spread to jmh thread benchmark start range, more comments to benchmarks parameters and purpose --- ...rallelMergeCombiningSequenceBenchmark.java | 39 +++++++++++++++---- ...rallelMergeCombiningSequenceBenchmark.java | 6 +++ ...eCombiningSequenceJmhThreadsBenchmark.java | 21 ++++++++++ 3 files changed, 59 insertions(+), 7 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index 08a9ce77b8c9..a221f72a3b64 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -70,6 +70,15 @@ public class BaseParallelMergeCombiningSequenceBenchmark }) int rowsPerSequence; + /** + * Strategy encodes the type of sequence and configuration parameters for that sequence. + * + * Strategies of the form: 'parallelism-{parallelism}-{targetTime}ms-{batchSize}-{yieldAfter}' + * encode the parameters for a {@link ParallelMergeCombiningSequence}. + * + * A strategy of: 'combiningMergeSequence-same-thread' (or an unrecognized value) will use a + * {@link CombiningSequence} that wraps a {@link MergeSequence} + */ @Param({ "combiningMergeSequence-same-thread", "parallelism-1-10ms-256-1024", @@ -88,12 +97,28 @@ public class BaseParallelMergeCombiningSequenceBenchmark String strategy; + /** + * This encodes the type of input sequences and parameters that control their behavior. + * 'non-blocking-sequence' uses {@link ParallelMergeCombiningSequenceTest#nonBlockingSequence} to as you might expect + * create an input sequence that is lazily generated and will not block while being consumed. + * + * 'initially-blocking-sequence-{startDelayStart}-{startDelayEnd}ms' uses + * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will + * initially block for a random time within the range specified in the parameter, and will not perform any additional + * blocking during further processing. + * + * 'blocking-sequence-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms' uses + * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will + * initially block for a random time within the range specified in the parameter, and additionally will randomly block + * up to the number of occurrences for up to the delay encoded in the parameter. + * + * note: beware when using the blocking sequences for a direct comparison between strategies + * at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable + * between strategies or number of sequences, much less between iterations of the same strategy, compensation in + * the form of running a lot of iterations could potentially make them more directly comparable + */ @Param({ "non-blocking-sequence", - // note: beware when using the blocking sequences for a direct comparison between strategies - // at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable - // between strategies or number of sequences, much less between iterations of the same strategy, compensation in - // the form of running a lot of iterations could potentially make them more directly comparable "initially-blocking-sequence-100-500ms", "initially-blocking-sequence-4000-5000ms", "blocking-sequence-10-100ms-10-1ms" @@ -166,7 +191,7 @@ public void setup() } - List> createInputSequences() + private List> createInputSequences() { List> inputSequences = new ArrayList<>(numSequences); for (int j = 0; j < numSequences; j++) { @@ -175,7 +200,7 @@ List> createInputSequences( return inputSequences; } - Sequence createParallelSequence() + private Sequence createParallelSequence() { return new ParallelMergeCombiningSequence<>( MERGE_POOL, @@ -192,7 +217,7 @@ Sequence createParallelSequence() ); } - Sequence createCombiningMergeSequence() + private Sequence createCombiningMergeSequence() { return CombiningSequence.create( new MergeSequence<>( diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java index 38acf2a9bf69..74907ab6e0eb 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java @@ -55,6 +55,9 @@ public class ParallelMergeCombiningSequenceBenchmark extends BaseParallelMergeCo // this should be as large as the largest value of concurrentSequenceConsumers private static final ExecutorService CONSUMER_POOL = Execs.multiThreaded(64, "mock-http-thread"); + /** + * Number of threads to run on {@link #CONSUMER_POOL}, each running {@link #consumeSequence} + */ @Param({ "1", "2", @@ -66,6 +69,9 @@ public class ParallelMergeCombiningSequenceBenchmark extends BaseParallelMergeCo }) private int concurrentSequenceConsumers; + /** + * Offset to start each thread of {@link #concurrentSequenceConsumers} + */ @Param({ "0", "10", diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java index fd2033aa001a..e496bdaab131 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java @@ -25,6 +25,7 @@ import org.openjdk.jmh.annotations.Measurement; import org.openjdk.jmh.annotations.Mode; import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Threads; @@ -55,6 +56,12 @@ @Measurement(iterations = 10) public class ParallelMergeCombiningSequenceJmhThreadsBenchmark extends BaseParallelMergeCombiningSequenceBenchmark { + /** + * each thread of benchmark method will delay randomly up to this many milliseconds + */ + @Param({"0", "20", "50"}) + public int startRangeMillis; + @Threads(1) @Benchmark @BenchmarkMode(Mode.AverageTime) @@ -127,6 +134,20 @@ public void execSixtyFour(Blackhole blackhole) consumeSequence(blackhole); } + @Override + void consumeSequence(Blackhole blackhole) + { + try { + if (startRangeMillis > 0) { + Thread.sleep(startRangeMillis); + } + super.consumeSequence(blackhole); + } + catch (InterruptedException iex) { + throw new RuntimeException(iex); + } + } + public static void main(String[] args) throws RunnerException { Options opt = new OptionsBuilder() From f8127f3726054febc551d408bdbcc93a0ab5f7c4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 31 Oct 2019 19:08:41 -0700 Subject: [PATCH 44/51] retool benchmark to allow modeling more typical heterogenous heavy workloads --- ...rallelMergeCombiningSequenceBenchmark.java | 113 +++++++++++++----- ...rallelMergeCombiningSequenceBenchmark.java | 4 +- ...eCombiningSequenceJmhThreadsBenchmark.java | 4 +- 3 files changed, 84 insertions(+), 37 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index a221f72a3b64..6ce35e7f993c 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -38,6 +38,8 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Function; import java.util.function.Supplier; @State(Scope.Benchmark) @@ -63,13 +65,6 @@ public class BaseParallelMergeCombiningSequenceBenchmark }) int numSequences; - @Param({ - "1000", - "75000", - "10000000" - }) - int rowsPerSequence; - /** * Strategy encodes the type of sequence and configuration parameters for that sequence. * @@ -99,29 +94,45 @@ public class BaseParallelMergeCombiningSequenceBenchmark /** * This encodes the type of input sequences and parameters that control their behavior. - * 'non-blocking-sequence' uses {@link ParallelMergeCombiningSequenceTest#nonBlockingSequence} to as you might expect - * create an input sequence that is lazily generated and will not block while being consumed. + * 'non-blocking-sequence-{numRows}' uses {@link ParallelMergeCombiningSequenceTest#nonBlockingSequence} to as you + * might expect create an input sequence that is lazily generated and will not block while being consumed. * - * 'initially-blocking-sequence-{startDelayStart}-{startDelayEnd}ms' uses + * 'initially-blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms' uses * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will * initially block for a random time within the range specified in the parameter, and will not perform any additional * blocking during further processing. * - * 'blocking-sequence-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms' uses + * 'blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms' uses * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will * initially block for a random time within the range specified in the parameter, and additionally will randomly block * up to the number of occurrences for up to the delay encoded in the parameter. * + * 'typical-distribution-sequence' will randomly produce a 'class' of input sequences at the following rates: + * - 80% probability of a small result set which has a short initial delay on the order of tens to hundreds of millis + * and input row counts of up to a few thousand + * - 20% probability produce a moderately large result set which has an initial delay in the range of a few seconds + * and input sequence row counts in the 50k-75k range + * This input sequence is only useful when testing a large number of concurrent threads + * * note: beware when using the blocking sequences for a direct comparison between strategies * at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable * between strategies or number of sequences, much less between iterations of the same strategy, compensation in * the form of running a lot of iterations could potentially make them more directly comparable */ @Param({ - "non-blocking-sequence", - "initially-blocking-sequence-100-500ms", - "initially-blocking-sequence-4000-5000ms", - "blocking-sequence-10-100ms-10-1ms" + "non-blocking-sequence-1000", + "non-blocking-sequence-75000", + "non-blocking-sequence-10000000", + "initially-blocking-sequence-1000-100-500ms", + "initially-blocking-sequence-75000-100-500ms", + "initially-blocking-sequence-10000000-100-500ms", + "initially-blocking-sequence-1000-4000-5000ms", + "initially-blocking-sequence-75000-4000-5000ms", + "initially-blocking-sequence-10000000-4000-5000ms", + "blocking-sequence-1000-10-500ms-10-1ms", + "blocking-sequence-75000-10-500ms-10-1ms", + "blocking-sequence-10000000-10-500ms-10-1ms", + "typical-distribution-sequence" }) String inputSequenceType; @@ -130,7 +141,8 @@ public class BaseParallelMergeCombiningSequenceBenchmark private int batchSize; private int yieldAfter; - private Supplier> inputSequenceFactory; + private Supplier> homogenousInputSequenceFactory; + private Function> randomInputSequenceFactory; private Supplier> outputSequenceFactory; @Setup(Level.Trial) @@ -138,14 +150,15 @@ public void setup() { String[] inputSequenceTypeSplit = inputSequenceType.split("-"); if ("initially".equals(inputSequenceTypeSplit[0])) { - // e.g. "initially-blocking-sequence-{startDelayStart}-{startDelayEnd}ms" - final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[3]); + // e.g. "initially-blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms" + final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); + final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[4]); final int startDelayEndMillis = Integer.parseInt( - inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2) + inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2) ); - inputSequenceFactory = () -> + homogenousInputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.blockingSequence( - rowsPerSequence, + numRows, startDelayStartMillis, startDelayEndMillis, -1, @@ -153,28 +166,57 @@ public void setup() true ); } else if ("blocking".equals(inputSequenceTypeSplit[0])) { - // e.g. "blocking-sequence-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms" - final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[2]); + // e.g. "blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms" + final int numRows = Integer.parseInt(inputSequenceTypeSplit[2]); + final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[3]); final int startDelayEndMillis = Integer.parseInt( - inputSequenceTypeSplit[3].substring(0, inputSequenceTypeSplit[3].length() - 2) + inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2) ); - final int numberOfTimesToBlock = Integer.parseInt(inputSequenceTypeSplit[4]); + final int numberOfTimesToBlock = Integer.parseInt(inputSequenceTypeSplit[5]); final int maxIterationDelayMillis = Integer.parseInt( - inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2) + inputSequenceTypeSplit[6].substring(0, inputSequenceTypeSplit[6].length() - 2) ); - final int frequency = rowsPerSequence / numberOfTimesToBlock; - inputSequenceFactory = () -> + final int frequency = numRows / numberOfTimesToBlock; + homogenousInputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.blockingSequence( - rowsPerSequence, + numRows, startDelayStartMillis, startDelayEndMillis, frequency, maxIterationDelayMillis, true ); - } else { // non-blocking sequence - inputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.nonBlockingSequence(rowsPerSequence, true); + } else if ("non".equals(inputSequenceTypeSplit[0])) { + // e.g. "non-blocking-sequence-{numRows}" + final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); + homogenousInputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.nonBlockingSequence(numRows, true); + } else { // "typical distribution" input sequence + // approximately 80% of threads will merge/combine small result sets between 500-10k results per input sequence + // blocking for 50-200 ms before initial results are yielded + // approximately 20% of threads will merge/combine moderate sized result sets between 50k-75k per input + // sequence, blocking for 1000-2500 ms before initial results are yielded + randomInputSequenceFactory = (d) -> { + if (d < 0.80) { // small queries + return ParallelMergeCombiningSequenceTest.blockingSequence( + ThreadLocalRandom.current().nextInt(500, 10000), + 50, + 200, + -1, + 0, + true + ); + } else { // moderately large queries + return ParallelMergeCombiningSequenceTest.blockingSequence( + ThreadLocalRandom.current().nextInt(50_000, 75_000), + 1000, + 2500, + -1, + 0, + true + ); + } + }; } String[] strategySplit = strategy.split("-"); @@ -194,8 +236,13 @@ public void setup() private List> createInputSequences() { List> inputSequences = new ArrayList<>(numSequences); + final double d = ThreadLocalRandom.current().nextDouble(0.0, 1.0); for (int j = 0; j < numSequences; j++) { - inputSequences.add(inputSequenceFactory.get()); + if (randomInputSequenceFactory != null) { + inputSequences.add(randomInputSequenceFactory.apply(d)); + } else { + inputSequences.add(homogenousInputSequenceFactory.get()); + } } return inputSequences; } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java index 74907ab6e0eb..dcac0e203fbe 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java @@ -47,8 +47,8 @@ @State(Scope.Benchmark) @Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") -@Warmup(iterations = 5) -@Measurement(iterations = 10) +@Warmup(iterations = 1) +@Measurement(iterations = 1) public class ParallelMergeCombiningSequenceBenchmark extends BaseParallelMergeCombiningSequenceBenchmark { private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java index e496bdaab131..5691bfbbd08b 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java @@ -48,12 +48,12 @@ * {@link ParallelMergeCombiningSequenceBenchmark} benchmark is also able to introduce a delay between the concurrent * processing threads to simulate other more realistic patterns than a simulataneous burst of concurrent threads. * - * Yes, this benchmark is sort of lame, unfortunately I could not figure out how to parameterize @Threads. + * Yes, this benchmark is sort of lame, unfortunately I could not figure out how to parameterize @Threads yet. */ @State(Scope.Benchmark) @Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") @Warmup(iterations = 5) -@Measurement(iterations = 10) +@Measurement(iterations = 25) public class ParallelMergeCombiningSequenceJmhThreadsBenchmark extends BaseParallelMergeCombiningSequenceBenchmark { /** From 0b1ba14ab67473d8e4c014f5c1bbe0099b8f525d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 31 Oct 2019 19:10:58 -0700 Subject: [PATCH 45/51] spelling --- ...arallelMergeCombiningSequenceBenchmark.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index 6ce35e7f993c..82b3222c6207 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -141,8 +141,8 @@ public class BaseParallelMergeCombiningSequenceBenchmark private int batchSize; private int yieldAfter; - private Supplier> homogenousInputSequenceFactory; - private Function> randomInputSequenceFactory; + private Supplier> homogeneousInputSequenceFactory; + private Function> heterogeneousInputSequenceFactory; private Supplier> outputSequenceFactory; @Setup(Level.Trial) @@ -156,7 +156,7 @@ public void setup() final int startDelayEndMillis = Integer.parseInt( inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2) ); - homogenousInputSequenceFactory = () -> + homogeneousInputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.blockingSequence( numRows, startDelayStartMillis, @@ -177,7 +177,7 @@ public void setup() inputSequenceTypeSplit[6].substring(0, inputSequenceTypeSplit[6].length() - 2) ); final int frequency = numRows / numberOfTimesToBlock; - homogenousInputSequenceFactory = () -> + homogeneousInputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.blockingSequence( numRows, startDelayStartMillis, @@ -189,14 +189,14 @@ public void setup() } else if ("non".equals(inputSequenceTypeSplit[0])) { // e.g. "non-blocking-sequence-{numRows}" final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); - homogenousInputSequenceFactory = () -> + homogeneousInputSequenceFactory = () -> ParallelMergeCombiningSequenceTest.nonBlockingSequence(numRows, true); } else { // "typical distribution" input sequence // approximately 80% of threads will merge/combine small result sets between 500-10k results per input sequence // blocking for 50-200 ms before initial results are yielded // approximately 20% of threads will merge/combine moderate sized result sets between 50k-75k per input // sequence, blocking for 1000-2500 ms before initial results are yielded - randomInputSequenceFactory = (d) -> { + heterogeneousInputSequenceFactory = (d) -> { if (d < 0.80) { // small queries return ParallelMergeCombiningSequenceTest.blockingSequence( ThreadLocalRandom.current().nextInt(500, 10000), @@ -238,10 +238,10 @@ private List> createInputSe List> inputSequences = new ArrayList<>(numSequences); final double d = ThreadLocalRandom.current().nextDouble(0.0, 1.0); for (int j = 0; j < numSequences; j++) { - if (randomInputSequenceFactory != null) { - inputSequences.add(randomInputSequenceFactory.apply(d)); + if (heterogeneousInputSequenceFactory != null) { + inputSequences.add(heterogeneousInputSequenceFactory.apply(d)); } else { - inputSequences.add(homogenousInputSequenceFactory.get()); + inputSequences.add(homogeneousInputSequenceFactory.get()); } } return inputSequences; From e5e2940d921a6a5d0e33e4e4722b75c2db822651 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 31 Oct 2019 21:10:54 -0700 Subject: [PATCH 46/51] fix --- .../BaseParallelMergeCombiningSequenceBenchmark.java | 2 +- .../sequences/ParallelMergeCombiningSequenceBenchmark.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index 82b3222c6207..b206eccacfa2 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -186,7 +186,7 @@ public void setup() maxIterationDelayMillis, true ); - } else if ("non".equals(inputSequenceTypeSplit[0])) { + } else if ("non".equals(inputSequenceTypeSplit[0])) { // e.g. "non-blocking-sequence-{numRows}" final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); homogeneousInputSequenceFactory = () -> diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java index dcac0e203fbe..599cbe8f3c43 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java @@ -47,8 +47,8 @@ @State(Scope.Benchmark) @Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") -@Warmup(iterations = 1) -@Measurement(iterations = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 25) public class ParallelMergeCombiningSequenceBenchmark extends BaseParallelMergeCombiningSequenceBenchmark { private static final Logger log = new Logger(ParallelMergeCombiningSequenceBenchmark.class); From 6b3baa6d1bc908e18459bc2ebd7c8eedffdedb21 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 1 Nov 2019 05:19:44 -0700 Subject: [PATCH 47/51] refactor benchmarks --- ...rallelMergeCombiningSequenceBenchmark.java | 207 ++++++------------ ...rallelMergeCombiningSequenceBenchmark.java | 124 ++++++++++- ...eCombiningSequenceJmhThreadsBenchmark.java | 163 -------------- ...rgeCombiningSequenceThreadedBenchmark.java | 60 +++++ 4 files changed, 250 insertions(+), 304 deletions(-) delete mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java index b206eccacfa2..14f240295752 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/BaseParallelMergeCombiningSequenceBenchmark.java @@ -91,134 +91,21 @@ public class BaseParallelMergeCombiningSequenceBenchmark }) String strategy; - - /** - * This encodes the type of input sequences and parameters that control their behavior. - * 'non-blocking-sequence-{numRows}' uses {@link ParallelMergeCombiningSequenceTest#nonBlockingSequence} to as you - * might expect create an input sequence that is lazily generated and will not block while being consumed. - * - * 'initially-blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms' uses - * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will - * initially block for a random time within the range specified in the parameter, and will not perform any additional - * blocking during further processing. - * - * 'blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms' uses - * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will - * initially block for a random time within the range specified in the parameter, and additionally will randomly block - * up to the number of occurrences for up to the delay encoded in the parameter. - * - * 'typical-distribution-sequence' will randomly produce a 'class' of input sequences at the following rates: - * - 80% probability of a small result set which has a short initial delay on the order of tens to hundreds of millis - * and input row counts of up to a few thousand - * - 20% probability produce a moderately large result set which has an initial delay in the range of a few seconds - * and input sequence row counts in the 50k-75k range - * This input sequence is only useful when testing a large number of concurrent threads - * - * note: beware when using the blocking sequences for a direct comparison between strategies - * at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable - * between strategies or number of sequences, much less between iterations of the same strategy, compensation in - * the form of running a lot of iterations could potentially make them more directly comparable - */ - @Param({ - "non-blocking-sequence-1000", - "non-blocking-sequence-75000", - "non-blocking-sequence-10000000", - "initially-blocking-sequence-1000-100-500ms", - "initially-blocking-sequence-75000-100-500ms", - "initially-blocking-sequence-10000000-100-500ms", - "initially-blocking-sequence-1000-4000-5000ms", - "initially-blocking-sequence-75000-4000-5000ms", - "initially-blocking-sequence-10000000-4000-5000ms", - "blocking-sequence-1000-10-500ms-10-1ms", - "blocking-sequence-75000-10-500ms-10-1ms", - "blocking-sequence-10000000-10-500ms-10-1ms", - "typical-distribution-sequence" - }) - String inputSequenceType; - private int parallelism; private int targetTaskTimeMillis; private int batchSize; private int yieldAfter; - private Supplier> homogeneousInputSequenceFactory; - private Function> heterogeneousInputSequenceFactory; - private Supplier> outputSequenceFactory; + private Function>, Sequence> outputSequenceFactory; @Setup(Level.Trial) public void setup() { - String[] inputSequenceTypeSplit = inputSequenceType.split("-"); - if ("initially".equals(inputSequenceTypeSplit[0])) { - // e.g. "initially-blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms" - final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); - final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[4]); - final int startDelayEndMillis = Integer.parseInt( - inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2) - ); - homogeneousInputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.blockingSequence( - numRows, - startDelayStartMillis, - startDelayEndMillis, - -1, - 0, - true - ); - } else if ("blocking".equals(inputSequenceTypeSplit[0])) { - // e.g. "blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms" - final int numRows = Integer.parseInt(inputSequenceTypeSplit[2]); - final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[3]); - final int startDelayEndMillis = Integer.parseInt( - inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2) - ); - final int numberOfTimesToBlock = Integer.parseInt(inputSequenceTypeSplit[5]); - final int maxIterationDelayMillis = Integer.parseInt( - inputSequenceTypeSplit[6].substring(0, inputSequenceTypeSplit[6].length() - 2) - ); - final int frequency = numRows / numberOfTimesToBlock; - homogeneousInputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.blockingSequence( - numRows, - startDelayStartMillis, - startDelayEndMillis, - frequency, - maxIterationDelayMillis, - true - ); - } else if ("non".equals(inputSequenceTypeSplit[0])) { - // e.g. "non-blocking-sequence-{numRows}" - final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); - homogeneousInputSequenceFactory = () -> - ParallelMergeCombiningSequenceTest.nonBlockingSequence(numRows, true); - } else { // "typical distribution" input sequence - // approximately 80% of threads will merge/combine small result sets between 500-10k results per input sequence - // blocking for 50-200 ms before initial results are yielded - // approximately 20% of threads will merge/combine moderate sized result sets between 50k-75k per input - // sequence, blocking for 1000-2500 ms before initial results are yielded - heterogeneousInputSequenceFactory = (d) -> { - if (d < 0.80) { // small queries - return ParallelMergeCombiningSequenceTest.blockingSequence( - ThreadLocalRandom.current().nextInt(500, 10000), - 50, - 200, - -1, - 0, - true - ); - } else { // moderately large queries - return ParallelMergeCombiningSequenceTest.blockingSequence( - ThreadLocalRandom.current().nextInt(50_000, 75_000), - 1000, - 2500, - -1, - 0, - true - ); - } - }; - } + setupOutputSequence(); + } + void setupOutputSequence() + { String[] strategySplit = strategy.split("-"); if ("parallelism".equals(strategySplit[0])) { // "parallelism-{parallelism}-{targetTime}ms-{batchSize}-{yieldAfter}" @@ -226,32 +113,20 @@ public void setup() targetTaskTimeMillis = Integer.parseInt(strategySplit[2].substring(0, strategySplit[2].length() - 2)); batchSize = Integer.parseInt(strategySplit[3]); yieldAfter = Integer.parseInt(strategySplit[4]); - outputSequenceFactory = () -> createParallelSequence(); + outputSequenceFactory = this::createParallelSequence; } else { - outputSequenceFactory = () -> createCombiningMergeSequence(); + outputSequenceFactory = this::createCombiningMergeSequence; } } - private List> createInputSequences() - { - List> inputSequences = new ArrayList<>(numSequences); - final double d = ThreadLocalRandom.current().nextDouble(0.0, 1.0); - for (int j = 0; j < numSequences; j++) { - if (heterogeneousInputSequenceFactory != null) { - inputSequences.add(heterogeneousInputSequenceFactory.apply(d)); - } else { - inputSequences.add(homogeneousInputSequenceFactory.get()); - } - } - return inputSequences; - } - - private Sequence createParallelSequence() + Sequence createParallelSequence( + List> inputSequences + ) { return new ParallelMergeCombiningSequence<>( MERGE_POOL, - createInputSequences(), + inputSequences, ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN, false, @@ -264,22 +139,25 @@ private Sequence createParallelSeque ); } - private Sequence createCombiningMergeSequence() + Sequence createCombiningMergeSequence( + List> inputSequences + ) { return CombiningSequence.create( new MergeSequence<>( ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, - Sequences.simple(createInputSequences()) + Sequences.simple(inputSequences) ), ParallelMergeCombiningSequenceTest.INT_PAIR_ORDERING, ParallelMergeCombiningSequenceTest.INT_PAIR_MERGE_FN ); } - void consumeSequence(Blackhole blackhole) + void consumeSequence(Blackhole blackhole, Supplier> supplier) { try { - Yielder yielder = Yielders.each(outputSequenceFactory.get()); + Yielder yielder = + Yielders.each(outputSequenceFactory.apply(createInputSequences(supplier))); ParallelMergeCombiningSequenceTest.IntPair prev; while (!yielder.isDone()) { @@ -293,4 +171,53 @@ void consumeSequence(Blackhole blackhole) throw new RuntimeException(anyException); } } + + List> createInputSequences( + Supplier> supplier + ) + { + List> inputSequences = new ArrayList<>(numSequences); + for (int j = 0; j < numSequences; j++) { + inputSequences.add(supplier.get()); + } + return inputSequences; + } + + Sequence generateSmallSequence() + { + return ParallelMergeCombiningSequenceTest.blockingSequence( + ThreadLocalRandom.current().nextInt(500, 10000), + 50, + 200, + -1, + 0, + true + ); + } + + Sequence generateModeratelyLargeSequence() + { + return ParallelMergeCombiningSequenceTest.blockingSequence( + ThreadLocalRandom.current().nextInt(50_000, 75_000), + 1000, + 2500, + -1, + 0, + true + ); + } + + Sequence generateLargeSequence() + { + final int numRows = ThreadLocalRandom.current().nextInt(1_500_000, 10_000_000); + final int frequency = numRows / 5; + return ParallelMergeCombiningSequenceTest.blockingSequence( + numRows, + 5000, + 10000, + frequency, + 10, + true + ); + } } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java index 599cbe8f3c43..be6b2a6291a6 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceBenchmark.java @@ -20,15 +20,19 @@ package org.apache.druid.benchmark.sequences; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequenceTest; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Measurement; import org.openjdk.jmh.annotations.Mode; import org.openjdk.jmh.annotations.OutputTimeUnit; import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; @@ -42,7 +46,10 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; @State(Scope.Benchmark) @@ -81,6 +88,114 @@ public class ParallelMergeCombiningSequenceBenchmark extends BaseParallelMergeCo }) private int concurrentConsumerDelayMillis; + /** + * This encodes the type of input sequences and parameters that control their behavior. + * 'non-blocking-sequence-{numRows}' uses {@link ParallelMergeCombiningSequenceTest#nonBlockingSequence} to as you + * might expect create an input sequence that is lazily generated and will not block while being consumed. + * + * 'initially-blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms' uses + * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will + * initially block for a random time within the range specified in the parameter, and will not perform any additional + * blocking during further processing. + * + * 'blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms' uses + * {@link ParallelMergeCombiningSequenceTest#blockingSequence} to create a lazily generated input sequence that will + * initially block for a random time within the range specified in the parameter, and additionally will randomly block + * up to the number of occurrences for up to the delay encoded in the parameter. + * + * 'typical-distribution-sequence' will randomly produce a 'class' of input sequences at the following rates: + * - 80% probability of a small result set which has a short initial delay on the order of tens to hundreds of millis + * and input row counts of up to a few thousand + * - 20% probability produce a moderately large result set which has an initial delay in the range of a few seconds + * and input sequence row counts in the 50k-75k range + * This input sequence is only useful when testing a large number of concurrent threads + * + * note: beware when using the blocking sequences for a direct comparison between strategies + * at minimum they are useful for gauging behavior when sequences block, but because the sequences are not stable + * between strategies or number of sequences, much less between iterations of the same strategy, compensation in + * the form of running a lot of iterations could potentially make them more directly comparable + */ + @Param({ + "non-blocking-sequence-1000", + "non-blocking-sequence-75000", + "non-blocking-sequence-10000000", + "initially-blocking-sequence-1000-100-500ms", + "initially-blocking-sequence-75000-100-500ms", + "initially-blocking-sequence-10000000-100-500ms", + "initially-blocking-sequence-1000-4000-5000ms", + "initially-blocking-sequence-75000-4000-5000ms", + "initially-blocking-sequence-10000000-4000-5000ms", + "blocking-sequence-1000-10-500ms-10-1ms", + "blocking-sequence-75000-10-500ms-10-1ms", + "blocking-sequence-10000000-10-500ms-10-1ms", + "typical-distribution-sequence" + }) + String inputSequenceType; + + private Supplier> homogeneousInputSequenceFactory; + private Function> heterogeneousInputSequenceFactory; + + @Setup(Level.Trial) + public void setupInputSequenceGenerator() + { + String[] inputSequenceTypeSplit = inputSequenceType.split("-"); + if ("initially".equals(inputSequenceTypeSplit[0])) { + // e.g. "initially-blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms" + final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); + final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[4]); + final int startDelayEndMillis = Integer.parseInt( + inputSequenceTypeSplit[5].substring(0, inputSequenceTypeSplit[5].length() - 2) + ); + homogeneousInputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.blockingSequence( + numRows, + startDelayStartMillis, + startDelayEndMillis, + -1, + 0, + true + ); + } else if ("blocking".equals(inputSequenceTypeSplit[0])) { + // e.g. "blocking-sequence-{numRows}-{startDelayStart}-{startDelayEnd}ms-{numberOfTimesToBlock}-{frequencyDelay}ms" + final int numRows = Integer.parseInt(inputSequenceTypeSplit[2]); + final int startDelayStartMillis = Integer.parseInt(inputSequenceTypeSplit[3]); + final int startDelayEndMillis = Integer.parseInt( + inputSequenceTypeSplit[4].substring(0, inputSequenceTypeSplit[4].length() - 2) + ); + final int numberOfTimesToBlock = Integer.parseInt(inputSequenceTypeSplit[5]); + final int maxIterationDelayMillis = Integer.parseInt( + inputSequenceTypeSplit[6].substring(0, inputSequenceTypeSplit[6].length() - 2) + ); + final int frequency = numRows / numberOfTimesToBlock; + homogeneousInputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.blockingSequence( + numRows, + startDelayStartMillis, + startDelayEndMillis, + frequency, + maxIterationDelayMillis, + true + ); + } else if ("non".equals(inputSequenceTypeSplit[0])) { + // e.g. "non-blocking-sequence-{numRows}" + final int numRows = Integer.parseInt(inputSequenceTypeSplit[3]); + homogeneousInputSequenceFactory = () -> + ParallelMergeCombiningSequenceTest.nonBlockingSequence(numRows, true); + } else { // "typical distribution" input sequence + // approximately 80% of threads will merge/combine small result sets between 500-10k results per input sequence + // blocking for 50-200 ms before initial results are yielded + // approximately 20% of threads will merge/combine moderate sized result sets between 50k-75k per input + // sequence, blocking for 1000-2500 ms before initial results are yielded + heterogeneousInputSequenceFactory = (d) -> { + if (d < 0.80) { // small queries + return generateSmallSequence(); + } else { // moderately large queries + return generateModeratelyLargeSequence(); + } + }; + } + } + @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) @@ -101,7 +216,14 @@ private List createConsumers(Blackhole blackhole, int consumers, int del if (delayMillis > 0) { Thread.sleep(delayMillis); } - futures.add(CONSUMER_POOL.submit(() -> consumeSequence(blackhole))); + if (heterogeneousInputSequenceFactory != null) { + double d = ThreadLocalRandom.current().nextDouble(0.0, 1.0); + futures.add( + CONSUMER_POOL.submit(() -> consumeSequence(blackhole, () -> heterogeneousInputSequenceFactory.apply(d))) + ); + } else { + futures.add(CONSUMER_POOL.submit(() -> consumeSequence(blackhole, homogeneousInputSequenceFactory))); + } } return futures; diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java deleted file mode 100644 index 5691bfbbd08b..000000000000 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceJmhThreadsBenchmark.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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.benchmark.sequences; - -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; -import org.openjdk.jmh.results.format.ResultFormatType; -import org.openjdk.jmh.runner.Runner; -import org.openjdk.jmh.runner.RunnerException; -import org.openjdk.jmh.runner.options.Options; -import org.openjdk.jmh.runner.options.OptionsBuilder; - -import java.util.concurrent.TimeUnit; - -/** - * This benchmark measures a slightly different thing than - * {@link ParallelMergeCombiningSequenceBenchmark}. While this benchmark measures the - * average time each thread takes to complete per iteration, the other measures the average time for _all_ threads to - * complete defined by {@link ParallelMergeCombiningSequenceBenchmark#concurrentSequenceConsumers} using an - * {@link java.util.concurrent.ExecutorService} thread pool. Additionally, the - * {@link ParallelMergeCombiningSequenceBenchmark} benchmark is also able to introduce a delay between the concurrent - * processing threads to simulate other more realistic patterns than a simulataneous burst of concurrent threads. - * - * Yes, this benchmark is sort of lame, unfortunately I could not figure out how to parameterize @Threads yet. - */ -@State(Scope.Benchmark) -@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") -@Warmup(iterations = 5) -@Measurement(iterations = 25) -public class ParallelMergeCombiningSequenceJmhThreadsBenchmark extends BaseParallelMergeCombiningSequenceBenchmark -{ - /** - * each thread of benchmark method will delay randomly up to this many milliseconds - */ - @Param({"0", "20", "50"}) - public int startRangeMillis; - - @Threads(1) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execOne(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(2) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execTwo(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(4) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execFour(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(8) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execEight(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(16) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execSixteen(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(24) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execTwentyFour(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(32) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execThirtyTwo(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Threads(64) - @Benchmark - @BenchmarkMode(Mode.AverageTime) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void execSixtyFour(Blackhole blackhole) - { - consumeSequence(blackhole); - } - - @Override - void consumeSequence(Blackhole blackhole) - { - try { - if (startRangeMillis > 0) { - Thread.sleep(startRangeMillis); - } - super.consumeSequence(blackhole); - } - catch (InterruptedException iex) { - throw new RuntimeException(iex); - } - } - - public static void main(String[] args) throws RunnerException - { - Options opt = new OptionsBuilder() - .include(ParallelMergeCombiningSequenceJmhThreadsBenchmark.class.getSimpleName()) - .forks(1) - .syncIterations(true) - .resultFormat(ResultFormatType.CSV) - .result("parallel-merge-combining-sequence-threads.csv") - .build(); - - new Runner(opt).run(); - } -} diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java new file mode 100644 index 000000000000..684e9c84316c --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java @@ -0,0 +1,60 @@ +/* + * 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.benchmark.sequences; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Group; +import org.openjdk.jmh.annotations.GroupThreads; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +@Warmup(iterations = 5) +@Measurement(iterations = 25) +public class ParallelMergeCombiningSequenceThreadedBenchmark extends BaseParallelMergeCombiningSequenceBenchmark +{ + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + @Group("consumers") + @GroupThreads(4) + public void consumeSmall(Blackhole blackhole) { + consumeSequence(blackhole, this::generateSmallSequence); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + @Group("consumers") + @GroupThreads(1) + public void consumeModeratelyLarge(Blackhole blackhole) { + consumeSequence(blackhole, this::generateModeratelyLargeSequence); + } +} From 7b8ebe904513b30698b14030234c716c04abf402 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 4 Nov 2019 00:06:06 -0800 Subject: [PATCH 48/51] formatting --- .../ParallelMergeCombiningSequenceThreadedBenchmark.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java index 684e9c84316c..ab4c0979518e 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java @@ -45,7 +45,8 @@ public class ParallelMergeCombiningSequenceThreadedBenchmark extends BaseParalle @OutputTimeUnit(TimeUnit.MILLISECONDS) @Group("consumers") @GroupThreads(4) - public void consumeSmall(Blackhole blackhole) { + public void consumeSmall(Blackhole blackhole) + { consumeSequence(blackhole, this::generateSmallSequence); } @@ -54,7 +55,8 @@ public void consumeSmall(Blackhole blackhole) { @OutputTimeUnit(TimeUnit.MILLISECONDS) @Group("consumers") @GroupThreads(1) - public void consumeModeratelyLarge(Blackhole blackhole) { + public void consumeModeratelyLarge(Blackhole blackhole) + { consumeSequence(blackhole, this::generateModeratelyLargeSequence); } } From dfc0560c405f766439b2131f7a9f892bf0ac275a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 4 Nov 2019 00:09:03 -0800 Subject: [PATCH 49/51] docs --- docs/querying/query-context.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md index 03b03d9df65a..573aec5a6716 100644 --- a/docs/querying/query-context.md +++ b/docs/querying/query-context.md @@ -41,10 +41,10 @@ The query context is used for various query configuration parameters. The follow |maxQueuedBytes | `druid.broker.http.maxQueuedBytes` | Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled.| |serializeDateTimeAsLong| `false` | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute process| |serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between Broker and compute process| -|enableParallelMerge|`true`|Enable parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| -|parallelMergeParallelism|`druid.processing.numMergePoolThreads`|Maximum number of parallel threads to use for parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| -|parallelMergeInitialYieldRows|`druid.processing.mergePoolTaskInitialYieldRows`|Number of rows to yield per ForkJoinPool merge task for parallel result merging on the Broker, before forking off a new task to continue merging sequences. See [Broker configuration](../configuration/index.html#broker) for more details.| -|parallelMergeSmallBatchRows|`druid.processing.mergePoolTaskInitialYieldRows`|Size of result batches to operate on in ForkJoinPool merge tasks for parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| +|enableParallelMerge|`true`|Enable parallel result merging on the Broker. Note that `druid.processing.merge.useParallelMergePool` must be enabled for this setting to be set to `true`. See [Broker configuration](../configuration/index.html#broker) for more details.| +|parallelMergeParallelism|`druid.processing.merge.pool.parallelism`|Maximum number of parallel threads to use for parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| +|parallelMergeInitialYieldRows|`druid.processing.merge.task.initialYieldNumRows`|Number of rows to yield per ForkJoinPool merge task for parallel result merging on the Broker, before forking off a new task to continue merging sequences. See [Broker configuration](../configuration/index.html#broker) for more details.| +|parallelMergeSmallBatchRows|`druid.processing.merge.task.smallBatchNumRows`|Size of result batches to operate on in ForkJoinPool merge tasks for parallel result merging on the Broker. See [Broker configuration](../configuration/index.html#broker) for more details.| In addition, some query types offer context parameters specific to that query type. From 1b2a81ce30aca29b56919ddd07c6b2c65e9407d5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 4 Nov 2019 13:20:31 -0800 Subject: [PATCH 50/51] add maxThreadStartDelay parameter to threaded benchmark --- ...rgeCombiningSequenceThreadedBenchmark.java | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java index ab4c0979518e..c10ece35f7f6 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java @@ -19,6 +19,8 @@ package org.apache.druid.benchmark.sequences; +import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequenceTest; +import org.apache.druid.java.util.common.guava.Sequence; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -27,12 +29,15 @@ import org.openjdk.jmh.annotations.Measurement; import org.openjdk.jmh.annotations.Mode; import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; @State(Scope.Benchmark) @Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") @@ -40,6 +45,13 @@ @Measurement(iterations = 25) public class ParallelMergeCombiningSequenceThreadedBenchmark extends BaseParallelMergeCombiningSequenceBenchmark { + @Param({ + "0", + "100", + "500" + }) + int maxThreadStartDelay; + @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) @@ -59,4 +71,18 @@ public void consumeModeratelyLarge(Blackhole blackhole) { consumeSequence(blackhole, this::generateModeratelyLargeSequence); } + + @Override + void consumeSequence(Blackhole blackhole, Supplier> supplier) + { + int delay = maxThreadStartDelay > 0 ? ThreadLocalRandom.current().nextInt(0, maxThreadStartDelay) : 0; + if (delay > 0) { + try { + Thread.sleep(delay); + } catch (InterruptedException ex) { + throw new RuntimeException(ex); + } + } + super.consumeSequence(blackhole, supplier); + } } From 51c1e3bd2345a05510b18606a4c80e9ee680e3f5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 5 Nov 2019 01:17:29 -0800 Subject: [PATCH 51/51] why does catch need to be on its own line but else doesnt --- .../ParallelMergeCombiningSequenceThreadedBenchmark.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java index c10ece35f7f6..1b932cae2c90 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/sequences/ParallelMergeCombiningSequenceThreadedBenchmark.java @@ -79,7 +79,8 @@ void consumeSequence(Blackhole blackhole, Supplier 0) { try { Thread.sleep(delay); - } catch (InterruptedException ex) { + } + catch (InterruptedException ex) { throw new RuntimeException(ex); } }