From 2aabb381ae7a1e8f90ca0623ab261406183cb9db Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 15 Nov 2018 13:09:42 -0800 Subject: [PATCH 1/5] Add support parallel combine in brokers --- .../CachingClusteredClientBenchmark.java | 701 ++++++++++++++++++ benchmarks/src/main/resources/log4j2.xml | 32 + .../druid/collections/BlockingPool.java | 13 + .../collections/DefaultBlockingPool.java | 30 +- .../druid/collections/DummyBlockingPool.java | 12 + .../collections/OrderedMergeSequence.java | 213 ------ .../druid/java/util/common/DateTimes.java | 5 + .../java/util/common/guava/BaseSequence.java | 5 +- .../util}/common/guava/CombiningSequence.java | 7 +- .../util/common/guava/LimitedSequence.java | 4 +- .../java/util/common/guava/MergeSequence.java | 30 +- .../guava/ParallelMergeCombineSequence.java | 314 ++++++++ .../java/util/common/guava/Sequence.java | 5 + .../util/common/guava/WrappingSequence.java | 6 +- .../query/AbstractPrioritizedCallable.java | 0 .../druid/query/DruidProcessingConfig.java | 0 .../apache/druid/query/ParallelCombines.java | 127 ++++ .../druid/query/PrioritizedCallable.java | 0 .../query/PrioritizedExecutorService.java | 0 .../druid/query/PrioritizedRunnable.java | 0 .../apache/druid/query/ThreadResource.java | 27 + .../druid/segment/column/ColumnConfig.java | 0 .../druid/collections/BlockingPoolTest.java | 32 +- .../collections/OrderedMergeSequenceTest.java | 350 --------- .../druid/common/utils/JodaUtilsTest.java | 1 - .../common/guava/CombiningSequenceTest.java | 6 +- .../common/guava/ComplexSequenceTest.java | 7 +- .../util/common/guava/MergeSequenceTest.java | 7 +- .../ParallelMergeCombineSequenceTest.java | 171 +++++ docs/content/querying/query-context.md | 2 + .../org/apache/druid/query/QueryContexts.java | 34 + .../apache/druid/query/QueryToolChest.java | 12 + .../druid/query/ResultMergeQueryRunner.java | 18 +- .../groupby/GroupByQueryQueryToolChest.java | 7 + .../epinephelinae/ParallelCombiner.java | 109 +-- .../query/groupby/orderby/TopNSequence.java | 24 +- .../groupby/strategy/GroupByStrategy.java | 12 + .../groupby/strategy/GroupByStrategyV2.java | 17 +- .../SegmentMetadataQueryQueryToolChest.java | 5 +- .../search/SearchQueryQueryToolChest.java | 21 +- .../select/SelectQueryQueryToolChest.java | 29 +- .../TimeseriesQueryQueryToolChest.java | 29 +- .../druid/query/topn/TopNQueryBuilder.java | 6 + .../query/topn/TopNQueryQueryToolChest.java | 35 +- .../groupby/GroupByQueryMergeBufferTest.java | 12 +- .../epinephelinae/ParallelCombinerTest.java | 4 +- .../spec/SpecificSegmentQueryRunnerTest.java | 2 +- .../apache/druid/client/BrokerServerView.java | 20 +- .../druid/client/CachingClusteredClient.java | 84 ++- .../client/ProcessingThreadResourcePool.java | 86 +++ .../druid/client/SimpleQueryRunner.java | 60 ++ .../org/apache/druid/client/cache/Cache.java | 9 +- ...ConnectionCountServerSelectorStrategy.java | 20 +- .../client/selector/QueryableDruidServer.java | 35 +- .../client/selector/RemoteDruidServer.java | 58 ++ ...chingClusteredClientFunctionalityTest.java | 41 +- ...usteredClientParallelMergeGroupByTest.java | 109 +++ ...gClusteredClientParallelMergeTestBase.java | 268 +++++++ ...eredClientParallelMergeTimeseriesTest.java | 96 +++ ...gClusteredClientParallelMergeTopNTest.java | 100 +++ .../client/CachingClusteredClientTest.java | 222 +++--- .../druid/client/DirectDruidClientTest.java | 10 +- .../apache/druid/client/TestQueryRunner.java | 46 ++ .../druid/client/TestTimelineServerView.java | 132 ++++ .../client/selector/ServerSelectorTest.java | 2 +- .../selector/TierSelectorStrategyTest.java | 22 +- 66 files changed, 2908 insertions(+), 995 deletions(-) create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java create mode 100644 benchmarks/src/main/resources/log4j2.xml delete mode 100644 core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java rename core/src/main/java/org/apache/druid/{ => java/util}/common/guava/CombiningSequence.java (95%) create mode 100644 core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java rename {processing => core}/src/main/java/org/apache/druid/query/AbstractPrioritizedCallable.java (100%) rename {processing => core}/src/main/java/org/apache/druid/query/DruidProcessingConfig.java (100%) create mode 100644 core/src/main/java/org/apache/druid/query/ParallelCombines.java rename {processing => core}/src/main/java/org/apache/druid/query/PrioritizedCallable.java (100%) rename {processing => core}/src/main/java/org/apache/druid/query/PrioritizedExecutorService.java (100%) rename {processing => core}/src/main/java/org/apache/druid/query/PrioritizedRunnable.java (100%) create mode 100644 core/src/main/java/org/apache/druid/query/ThreadResource.java rename {processing => core}/src/main/java/org/apache/druid/segment/column/ColumnConfig.java (100%) delete mode 100644 core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java rename core/src/test/java/org/apache/druid/{ => java/util}/common/guava/CombiningSequenceTest.java (96%) rename core/src/test/java/org/apache/druid/{ => java/util}/common/guava/ComplexSequenceTest.java (89%) create mode 100644 core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java create mode 100644 server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java create mode 100644 server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java create mode 100644 server/src/main/java/org/apache/druid/client/selector/RemoteDruidServer.java create mode 100644 server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java create mode 100644 server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java create mode 100644 server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java create mode 100644 server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java create mode 100644 server/src/test/java/org/apache/druid/client/TestQueryRunner.java create mode 100644 server/src/test/java/org/apache/druid/client/TestTimelineServerView.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 new file mode 100644 index 000000000000..2e036f5ad220 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -0,0 +1,701 @@ +/* + * 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.query; + +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; +import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; +import org.apache.druid.benchmark.datagen.BenchmarkSchemas; +import org.apache.druid.client.CachingClusteredClient; +import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.SimpleQueryRunner; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.ForegroundCachePopulator; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; +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.client.selector.TierSelectorStrategy; +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.DefaultBlockingPool; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.collections.StupidPool; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Druids; +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; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryEngine; +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.strategy.GroupByStrategySelector; +import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; +import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.query.topn.TopNQueryConfig; +import org.apache.druid.query.topn.TopNQueryQueryToolChest; +import org.apache.druid.query.topn.TopNQueryRunnerFactory; +import org.apache.druid.query.topn.TopNResultValue; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.NumberedShardSpec; +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.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +@State(Scope.Benchmark) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +@Warmup(iterations = 15) +@Measurement(iterations = 30) +public class CachingClusteredClientBenchmark +{ + @Param({"8"}) + private int numServers; + + @Param({"4", "2", "1"}) + private int numProcessingThreads; + + @Param({"75000"}) + private int rowsPerSegment; + + @Param({"all"}) + private String queryGranularity; + + @Param({"5120", "10240", "20480"}) + private int brokerParallelMergeQueueSize; + + private static final Logger log = new Logger(CachingClusteredClientBenchmark.class); + private static final String DATA_SOURCE = "ds"; + private static final int RNG_SEED = 9999; + private static final IndexMergerV9 INDEX_MERGER_V9; + private static final IndexIO INDEX_IO; + public static final ObjectMapper JSON_MAPPER; + + private Map queryableIndexes; + + private SimpleServerView serverView; + private QueryToolChestWarehouse toolChestWarehouse; + private QueryRunnerFactoryConglomerate conglomerate; + private CachingClusteredClient cachingClusteredClient; + private ExecutorService processingPool; + + private BenchmarkSchemaInfo schemaInfo; + private File tmpDir; + + private Query query; + + static { + JSON_MAPPER = new DefaultObjectMapper(); + JSON_MAPPER.setInjectableValues( + new Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), JSON_MAPPER) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) + ); + INDEX_IO = new IndexIO( + JSON_MAPPER, + () -> 0 + ); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); + } + + private void setupTimeseriesQuery() + { + BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList(basicSchema.getDataInterval()) + ); + + query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(intervalSpec) + .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) + .granularity(Granularity.fromString(queryGranularity)) + .context( + ImmutableMap.of( + QueryContexts.NUM_BROKER_PARALLEL_COMBINE_THREADS, + QueryContexts.NUM_CURRENT_AVAILABLE_THREADS, + QueryContexts.BROKER_PARALLEL_COMBINE_QUEUE_SIZE, + brokerParallelMergeQueueSize + ) + ) + .build(); + } + + private void setupTopNQuery() + { + BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList(basicSchema.getDataInterval()) + ); + + query = new TopNQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(intervalSpec) + .dimension(new DefaultDimensionSpec("dimUniform", null)) + .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) + .granularity(Granularity.fromString(queryGranularity)) + .metric("sumLongSequential") + .threshold(20480) + .context( + ImmutableMap.of( + QueryContexts.NUM_BROKER_PARALLEL_COMBINE_THREADS, + QueryContexts.NUM_CURRENT_AVAILABLE_THREADS, + QueryContexts.BROKER_PARALLEL_COMBINE_QUEUE_SIZE, + brokerParallelMergeQueueSize + ) + ) + .build(); + } + + private void setupGroupByQuery() + { + BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList(basicSchema.getDataInterval()) + ); + + query = GroupByQuery + .builder() + .setDataSource(DATA_SOURCE) + .setQuerySegmentSpec(intervalSpec) + .setDimensions( + new DefaultDimensionSpec("dimUniform", null), + new DefaultDimensionSpec("dimZipf", null) + ) + .setAggregatorSpecs(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) + .setGranularity(Granularity.fromString(queryGranularity)) + .setContext( + ImmutableMap.of( + QueryContexts.NUM_BROKER_PARALLEL_COMBINE_THREADS, + QueryContexts.NUM_CURRENT_AVAILABLE_THREADS, + QueryContexts.BROKER_PARALLEL_COMBINE_QUEUE_SIZE, + brokerParallelMergeQueueSize + ) + ) + .build(); + } + + @Setup(Level.Trial) + public void setup() throws IOException + { + final String schemaName = "basic"; + + schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + + final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator( + schemaInfo.getColumnSchemas(), + RNG_SEED + 1, + schemaInfo.getDataInterval(), + rowsPerSegment + ); + + tmpDir = Files.createTempDir(); + queryableIndexes = new HashMap<>(numServers); + + for (int i = 0; i < numServers; i++) { + final IncrementalIndex index = makeIncIndex(schemaInfo.isWithRollup()); + + for (int j = 0; j < rowsPerSegment; j++) { + final InputRow row = dataGenerator.nextRow(); + if (j % 20000 == 0) { + log.info("%,d/%,d rows generated.", i * rowsPerSegment + j, rowsPerSegment * numServers); + } + index.add(row); + } + + log.info( + "%,d/%,d rows generated, persisting segment %d/%d.", + (i + 1) * rowsPerSegment, + rowsPerSegment * numServers, + i + 1, + numServers + ); + + final File file = INDEX_MERGER_V9.persist( + index, + new File(tmpDir, String.valueOf(i)), + new IndexSpec(), + null + ); + + final QueryableIndex queryableIndex = INDEX_IO.loadIndex(file); + queryableIndexes.put(fromQueryableIndex(queryableIndex, file.length(), i), queryableIndex); + + index.close(); + } + + final DruidProcessingConfig processingConfig = new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + // Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby. + // Two buffers for the broker and one for the queryable + return 3; + } + + @Override + public int getNumThreads() + { + return numProcessingThreads; + } + }; + + conglomerate = new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>builder() + .put( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TopNQuery.class, + new TopNQueryRunnerFactory( + new StupidPool<>( + "TopNQueryRunnerFactory-bufferPool", + () -> ByteBuffer.allocate(10 * 1024 * 1024) + ), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + GroupByQuery.class, + makeGroupByQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + }, + processingConfig + ) + ) + .build() + ); + + toolChestWarehouse = new QueryToolChestWarehouse() + { + @Override + public > QueryToolChest getToolChest(final QueryType query) + { + return conglomerate.findFactory(query).getToolchest(); + } + }; + + serverView = new SimpleServerView(); + int serverSuffx = 1; + for (Entry entry : queryableIndexes.entrySet()) { + serverView.addServer( + createServer(serverSuffx++), + entry.getKey(), + entry.getValue() + ); + } + + processingPool = Execs.multiThreaded(processingConfig.getNumThreads(), "caching-clustered-client-benchmark"); + cachingClusteredClient = new CachingClusteredClient( + toolChestWarehouse, + serverView, + MapCache.create(0), + JSON_MAPPER, + new ForegroundCachePopulator(JSON_MAPPER, new CachePopulatorStats(), 0), + new CacheConfig(), + new DruidHttpClientConfig(), + processingPool, + processingConfig + ); + } + + private static GroupByQueryRunnerFactory makeGroupByQueryRunnerFactory( + final ObjectMapper mapper, + final GroupByQueryConfig config, + final DruidProcessingConfig processingConfig + ) + { + final Supplier configSupplier = Suppliers.ofInstance(config); + final NonBlockingPool bufferPool = new StupidPool<>( + "GroupByQueryEngine-bufferPool", + () -> ByteBuffer.allocateDirect(processingConfig.intermediateComputeSizeBytes()) + ); + final BlockingPool mergeBufferPool = new DefaultBlockingPool<>( + () -> ByteBuffer.allocateDirect(processingConfig.intermediateComputeSizeBytes()), + processingConfig.getNumMergeBuffers() + ); + final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + configSupplier, + new GroupByStrategyV1( + configSupplier, + new GroupByQueryEngine(configSupplier, bufferPool), + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + bufferPool + ), + new GroupByStrategyV2( + processingConfig, + configSupplier, + bufferPool, + mergeBufferPool, + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + strategySelector, + QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + ); + return new GroupByQueryRunnerFactory( + strategySelector, + toolChest + ); + } + + @TearDown(Level.Trial) + public void tearDown() + { + try { + if (queryableIndexes != null) { + for (QueryableIndex index : queryableIndexes.values()) { + index.close(); + } + } + + if (tmpDir != null) { + FileUtils.deleteDirectory(tmpDir); + } + processingPool.shutdown(); + } + catch (IOException e) { + log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir); + throw Throwables.propagate(e); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void timeseriesQuery(Blackhole blackhole) + { + setupTimeseriesQuery(); + final List> results = runQuery(); + + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void topNQuery(Blackhole blackhole) + { + setupTopNQuery(); + final List> results = runQuery(); + + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void groupByQuery(Blackhole blackhole) + { + setupGroupByQuery(); + final List results = runQuery(); + + for (Row result : results) { + blackhole.consume(result); + } + } + + private List runQuery() + { + //noinspection unchecked + QueryRunner theRunner = new FluentQueryRunnerBuilder<>(toolChestWarehouse.getToolChest(query)) + .create(cachingClusteredClient.getQueryRunnerForIntervals(query, query.getIntervals())) + .applyPreMergeDecoration() + .mergeResults() + .applyPostMergeDecoration(); + + //noinspection unchecked + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>()); + + return queryResult.toList(); + } + + private IncrementalIndex makeIncIndex(boolean withRollup) + { + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(withRollup) + .build() + ) + .setReportParseExceptions(false) + .setConcurrentEventAdd(true) + .setMaxRowCount(rowsPerSegment) + .buildOnheap(); + } + + private class SimpleServerView implements TimelineServerView + { + private final TierSelectorStrategy tierSelectorStrategy = new HighestPriorityTierSelectorStrategy( + new RandomServerSelectorStrategy() + ); + // server -> queryRunner + private final Map servers = new HashMap<>(); + // segmentId -> serverSelector + private final Map selectors = new HashMap<>(); + // dataSource -> version -> serverSelector + private final Map> timelines = new HashMap<>(); + + void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex) + { + servers.put( + server, + new SingleSegmentDruidServer( + server, + new SimpleQueryRunner( + conglomerate, + dataSegment.getIdentifier(), + queryableIndex + ) + ) + ); + addSegmentToServer(server, dataSegment); + } + + void addSegmentToServer(DruidServer server, DataSegment segment) + { + final ServerSelector selector = selectors.computeIfAbsent( + segment.getIdentifier(), + k -> new ServerSelector(segment, tierSelectorStrategy) + ); + selector.addServerAndUpdateSegment(servers.get(server), segment); + timelines.computeIfAbsent(segment.getDataSource(), k -> new VersionedIntervalTimeline<>(Ordering.natural())) + .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); + } + + @Nullable + @Override + public TimelineLookup getTimeline(DataSource dataSource) + { + final String table = Iterables.getOnlyElement(dataSource.getNames()); + return timelines.get(table); + } + + @Override + public List getDruidServers() + { + return Collections.emptyList(); + } + + @Override + public QueryRunner getQueryRunner(DruidServer server) + { + final SingleSegmentDruidServer queryableDruidServer = Preconditions.checkNotNull(servers.get(server), "server"); + return (QueryRunner) queryableDruidServer.getQueryRunner(); + } + + @Override + public void registerTimelineCallback(Executor exec, TimelineCallback callback) + { + // do nothing + } + + @Override + public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback callback) + { + // do nothing + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + // do nothing + } + } + + private class SingleSegmentDruidServer implements QueryableDruidServer + { + private final DruidServer server; + private final SimpleQueryRunner runner; + + SingleSegmentDruidServer(DruidServer server, SimpleQueryRunner runner) + { + this.server = server; + this.runner = runner; + } + + @Override + public DruidServer getServer() + { + return server; + } + + @Override + public SimpleQueryRunner getQueryRunner() + { + return runner; + } + } + + private static DruidServer createServer(int nameSuiffix) + { + return new DruidServer( + "server_" + nameSuiffix, + "127.0.0." + nameSuiffix, + null, + 10240L, + ServerType.HISTORICAL, + "default", + 0 + ); + } + + private static DataSegment fromQueryableIndex(QueryableIndex queryableIndex, long size, int seq) + { + return new DataSegment( + DATA_SOURCE, + queryableIndex.getDataInterval(), + "version", + Collections.emptyMap(), + Lists.newArrayList(queryableIndex.getAvailableDimensions().iterator()), + Arrays.stream(queryableIndex.getMetadata().getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()), + new NumberedShardSpec(seq, 0), + 0, + size + ); + } +} diff --git a/benchmarks/src/main/resources/log4j2.xml b/benchmarks/src/main/resources/log4j2.xml new file mode 100644 index 000000000000..dbce142e7f60 --- /dev/null +++ b/benchmarks/src/main/resources/log4j2.xml @@ -0,0 +1,32 @@ + + + + + + + + + + + + + + + diff --git a/core/src/main/java/org/apache/druid/collections/BlockingPool.java b/core/src/main/java/org/apache/druid/collections/BlockingPool.java index 91c3b35b6549..1c698cc93981 100644 --- a/core/src/main/java/org/apache/druid/collections/BlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/BlockingPool.java @@ -24,8 +24,21 @@ public interface BlockingPool { + /** + * Returns the total pool size. + */ int maxSize(); + /** + * Returns the number of current available resources. + */ + int available(); + + /** + * Poll all available resources from the pool. If there's no available resource, it returns an empty list. + */ + List> pollAll(); + /** * Take a resource from the pool, waiting up to the * specified wait time if necessary for an element to become available. diff --git a/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java index b1b43fd22477..56a9edbef07e 100644 --- a/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java @@ -70,10 +70,34 @@ public int maxSize() return maxSize; } - @VisibleForTesting - public int getPoolSize() + @Override + public int available() { - return objects.size(); + final ReentrantLock lock = this.lock; + lock.lock(); + try { + return objects.size(); + } + finally { + lock.unlock(); + } + } + + @Override + public List> pollAll() + { + checkInitialized(); + final ReentrantLock lock = this.lock; + lock.lock(); + try { + final List list = new ArrayList<>(objects.size()); + list.addAll(objects); + objects.clear(); + return list.stream().map(this::wrapObject).collect(Collectors.toList()); + } + finally { + lock.unlock(); + } } @Override diff --git a/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java index 037a489e1f19..1e41858d9d40 100644 --- a/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java @@ -44,6 +44,18 @@ public int maxSize() return 0; } + @Override + public int available() + { + return 0; + } + + @Override + public List> pollAll() + { + throw new UnsupportedOperationException(); + } + @Override public ReferenceCountingResourceHolder take(long timeoutMs) { diff --git a/core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java b/core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java deleted file mode 100644 index 7c861a9b5c25..000000000000 --- a/core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java +++ /dev/null @@ -1,213 +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.collections; - -import com.google.common.base.Function; -import com.google.common.base.Throwables; -import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.CloseQuietly; -import org.apache.druid.java.util.common.guava.Sequence; -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.guava.YieldingAccumulator; -import org.apache.druid.java.util.common.guava.YieldingAccumulators; -import org.apache.druid.java.util.common.io.Closer; - -import java.io.IOException; -import java.util.PriorityQueue; - -/** - * An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming - * that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators. - * - * Normally a merge operation could operate with the actual input iterators in any order as long as the actual values - * in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators - * be provided in the order of the first element of each iterator. - * - * If this doesn't make sense, check out OrderedMergeSequenceTest.testScrewsUpOnOutOfOrderBeginningOfList() - * - * It places this extra restriction on the input data in order to implement an optimization that allows it to - * remain as lazy as possible in the face of a common case where the iterators are just appended one after the other. - */ -public class OrderedMergeSequence implements Sequence -{ - private final Ordering ordering; - private final Sequence> sequences; - - public OrderedMergeSequence( - final Ordering ordering, - Sequence> sequences - ) - { - this.ordering = ordering; - this.sequences = sequences; - } - - @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) - { - Yielder yielder = null; - try { - yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); - return yielder.get(); - } - finally { - CloseQuietly.close(yielder); - } - } - - @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) - { - PriorityQueue> pQueue = new PriorityQueue>( - 32, - ordering.onResultOf( - new Function, T>() - { - @Override - public T apply(Yielder input) - { - return input.get(); - } - } - ) - ); - - Yielder> oldDudeAtCrosswalk = sequences.toYielder( - null, - new YieldingAccumulator, Sequence>() - { - @Override - public Yielder accumulate(Yielder accumulated, Sequence in) - { - final Yielder retVal = in.toYielder( - null, - new YieldingAccumulator() - { - @Override - public T accumulate(T accumulated, T in) - { - yield(); - return in; - } - } - ); - - if (retVal.isDone()) { - try { - retVal.close(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - return null; - } else { - yield(); - } - - return retVal; - } - } - ); - - return makeYielder(pQueue, oldDudeAtCrosswalk, initValue, accumulator); - } - - private Yielder makeYielder( - final PriorityQueue> pQueue, - Yielder> oldDudeAtCrosswalk, - OutType initVal, - final YieldingAccumulator accumulator - ) - { - OutType retVal = initVal; - while (!accumulator.yielded() && (!pQueue.isEmpty() || !oldDudeAtCrosswalk.isDone())) { - Yielder yielder; - if (oldDudeAtCrosswalk.isDone()) { - yielder = pQueue.remove(); - } else if (pQueue.isEmpty()) { - yielder = oldDudeAtCrosswalk.get(); - oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); - } else { - Yielder queueYielder = pQueue.peek(); - Yielder iterYielder = oldDudeAtCrosswalk.get(); - - if (ordering.compare(queueYielder.get(), iterYielder.get()) <= 0) { - yielder = pQueue.remove(); - } else { - yielder = oldDudeAtCrosswalk.get(); - oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); - } - } - - retVal = accumulator.accumulate(retVal, yielder.get()); - yielder = yielder.next(null); - if (yielder.isDone()) { - try { - yielder.close(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } else { - pQueue.add(yielder); - } - } - - if (!accumulator.yielded()) { - return Yielders.done(retVal, oldDudeAtCrosswalk); - } - - final OutType yieldVal = retVal; - final Yielder> finalOldDudeAtCrosswalk = oldDudeAtCrosswalk; - return new Yielder() - { - @Override - public OutType get() - { - return yieldVal; - } - - @Override - public Yielder next(OutType initValue) - { - accumulator.reset(); - return makeYielder(pQueue, finalOldDudeAtCrosswalk, initValue, accumulator); - } - - @Override - public boolean isDone() - { - return false; - } - - @Override - public void close() throws IOException - { - Closer closer = Closer.create(); - while (!pQueue.isEmpty()) { - closer.register(pQueue.remove()); - } - closer.close(); - } - }; - } -} diff --git a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java index f9a708d7e3b5..bdb783e0cde4 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java +++ b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java @@ -99,6 +99,11 @@ public static DateTime of(String instant) return new DateTime(instant, ISOChronology.getInstanceUTC()); } + public static DateTime of(String format, Object... formatArgs) + { + return of(StringUtils.format(format, formatArgs)); + } + public static DateTime of( int year, int monthOfYear, diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java index a11d742342f0..d3fdb93ad158 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java @@ -40,9 +40,10 @@ public BaseSequence( public OutType accumulate(OutType initValue, final Accumulator fn) { IterType iterator = maker.make(); + OutType accumulated = initValue; try { while (iterator.hasNext()) { - initValue = fn.accumulate(initValue, iterator.next()); + accumulated = fn.accumulate(accumulated, iterator.next()); } } catch (Throwable t) { @@ -55,7 +56,7 @@ public OutType accumulate(OutType initValue, final Accumulator extends YieldingSequenceBase @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - final LimitedYieldingAccumulator limitedAccumulator = new LimitedYieldingAccumulator<>( - accumulator - ); + final LimitedYieldingAccumulator limitedAccumulator = new LimitedYieldingAccumulator<>(accumulator); final Yielder subYielder = baseSequence.toYielder(initValue, limitedAccumulator); return new LimitedYielder<>(subYielder, limitedAccumulator); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java index 287fc62e289c..7182ede29f1c 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java @@ -46,22 +46,20 @@ public MergeSequence( @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - PriorityQueue> pQueue = new PriorityQueue<>( - 32, - ordering.onResultOf( - new Function, T>() - { - @Override - public T apply(Yielder input) - { - return input.get(); - } - } - ) - ); - - pQueue = baseSequences.accumulate( - pQueue, + PriorityQueue> pQueue = baseSequences.accumulate( + new PriorityQueue<>( + 32, + ordering.onResultOf( + new Function, T>() + { + @Override + public T apply(Yielder input) + { + return input.get(); + } + } + ) + ), (queue, in) -> { final Yielder yielder = in.toYielder( null, diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java new file mode 100644 index 000000000000..97c583f5a95c --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java @@ -0,0 +1,314 @@ +/* + * 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.collections.ReferenceCountingResourceHolder; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker; +import org.apache.druid.java.util.common.guava.nary.BinaryFn; +import org.apache.druid.query.ParallelCombines; +import org.apache.druid.query.PrioritizedRunnable; +import org.apache.druid.query.ThreadResource; + +import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; +import java.util.stream.IntStream; + +/** + * This sequence merges underlying {@link #baseSequences} and combines (aggregates) them in parallel. + * It creates a tree to merge and combine input streams in parallel, consisting of processing threads connected by + * blocking queues. In leaf nodes, the processing threads combines values from the {@link #baseSequences} and stores + * aggregates in its blocking queue. Intermediate threads reads values from the blocking queue of their children and + * stores computed aggregates its blocking queue. Finally, the caller thread of this class (usually it's an http thread + * in query processing) reads values from the queue of the root thread. Filling blocking queue and reading from it are + * done asynchronously. + */ +public class ParallelMergeCombineSequence extends YieldingSequenceBase +{ + private static final int MINIMUM_LEAF_COMBINE_DEGREE = 2; + + private final ExecutorService exec; + private final List> baseSequences; + private final Ordering ordering; + private final BinaryFn combineFn; + private final List> processingThreadHolders; + private final int queueSize; + private final boolean hasTimeout; + private final long timeoutAt; + private final int queryPriority; + + public ParallelMergeCombineSequence( + ExecutorService exec, + List> baseSequences, + Ordering ordering, + BinaryFn combineFn, + List> processingThreadHolders, + int queueSize, + boolean hasTimeout, + long timeout, + int queryPriority + ) + { + this.exec = exec; + this.baseSequences = (List>) baseSequences; + this.ordering = ordering; + this.combineFn = combineFn; + this.processingThreadHolders = processingThreadHolders; + this.queueSize = queueSize; + this.hasTimeout = hasTimeout; + this.timeoutAt = System.currentTimeMillis() + timeout; + this.queryPriority = queryPriority; + } + + @Override + public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + { + final Pair combineDegreeAndNumThreads = findCombineDegreeAndNumThreads( + processingThreadHolders.size(), + baseSequences.size() + ); + final int combineDegree = combineDegreeAndNumThreads.lhs; + final int numThreads = combineDegreeAndNumThreads.rhs; + + // Early release unnecessary processing threads + IntStream.range(0, processingThreadHolders.size() - numThreads) + .forEach(i -> processingThreadHolders.remove(0).close()); + + final Supplier> processingThreadSupplier = + new Supplier>() + { + private int next = 0; + + @Override + public ReferenceCountingResourceHolder get() + { + if (next < processingThreadHolders.size()) { + return processingThreadHolders.get(next++); + } else { + throw new ISE( + "WTH? current pointer[%d] is larger than available threads[%d]", + next, + processingThreadHolders.size() + ); + } + } + }; + + // In the result combine tree, nodes and edges are processing threads and blocking queues, respectively. + // In the leaf nodes, processing threads read data from historicals and fill its blocking queue. + // In the intermediate nodes, processing threads read data from the blocking queues of their children and fill + // its blocking queue. + // The caller thread reads the blocking queue of the root node. + final Pair, List> rootAndFutures = ParallelCombines.buildCombineTree( + baseSequences, + combineDegree, + combineDegree, + sequences -> runCombine(processingThreadSupplier.get(), sequences) + ); + + // Ignore futures since they are handled in the generated BaseSequence.IteratorMaker.cleanup(). + return rootAndFutures.lhs.toYielder(initValue, accumulator); + } + + private static Pair findCombineDegreeAndNumThreads(int numAvailableThreads, int numLeafNodes) + { + for (int combineDegree = MINIMUM_LEAF_COMBINE_DEGREE; combineDegree <= numLeafNodes; combineDegree++) { + final int numRequiredThreads = computeNumRequiredThreads(numLeafNodes, combineDegree); + if (numRequiredThreads <= numAvailableThreads) { + return Pair.of(combineDegree, numRequiredThreads); + } + } + + throw new ISE( + "Cannot find a proper combine degree for the combining tree. " + + "Each node of the combining tree requires a single thread. " + + "Try increasing druid.processing.numThreads or " + + "reducing numBrokerParallelCombineThreads[%d] in the query context for a smaller tree", + numAvailableThreads + ); + } + + private static int computeNumRequiredThreads(int numChildNodes, int combineDegree) + { + // numChildNodes is used to determine that the last node is needed for the current level. + // Please see ParallelCombines.buildCombineTree() for more details. + final int numChildrenForLastNode = numChildNodes % combineDegree; + final int numCurLevelNodes = numChildNodes / combineDegree + (numChildrenForLastNode > 1 ? 1 : 0); + final int numChildOfParentNodes = numCurLevelNodes + (numChildrenForLastNode == 1 ? 1 : 0); + + if (numChildOfParentNodes == 1) { + return numCurLevelNodes; + } else { + return numCurLevelNodes + + computeNumRequiredThreads(numChildOfParentNodes, combineDegree); + } + } + + private Pair, Future> runCombine( + ReferenceCountingResourceHolder processingThreadHolder, + List> sequenceList + ) + { + final Sequence> sequences = Sequences.simple(sequenceList); + final CombiningSequence combiningSequence = CombiningSequence.create( + new MergeSequence<>(ordering, sequences), + ordering, + combineFn + ); + + final BlockingQueue queue = new ArrayBlockingQueue<>(queueSize); + + final Future future = exec.submit( + new PrioritizedRunnable() + { + @Override + public int getPriority() + { + return queryPriority; + } + + @Override + public void run() + { + try { + combiningSequence.accumulate( + queue, + (theQueue, v) -> { + try { + addToQueue(theQueue, new ValueHolder(v)); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + return theQueue; + } + ); + // add a null to indicate this is the last one + addToQueue(queue, new ValueHolder(null)); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + finally { + processingThreadHolder.close(); + } + } + } + ); + + final Sequence backgroundCombineSequence = new BaseSequence<>( + new IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + private T nextVal; + + @Override + public boolean hasNext() + { + try { + final ValueHolder holder; + if (!hasTimeout) { + holder = queue.take(); + } else { + final long timeout = timeoutAt - System.currentTimeMillis(); + holder = queue.poll(timeout, TimeUnit.MILLISECONDS); + } + + if (holder == null) { + throw new RuntimeException(new TimeoutException()); + } + nextVal = holder.val; + return nextVal != null; + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public T next() + { + if (nextVal == null) { + throw new NoSuchElementException(); + } + return nextVal; + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + try { + if (future.isDone()) { + future.get(); + } else { + future.cancel(true); + } + } + catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + } + ); + + return Pair.of(backgroundCombineSequence, future); + } + + private void addToQueue(BlockingQueue queue, ValueHolder holder) + throws InterruptedException + { + if (!hasTimeout) { + queue.put(holder); + } else { + final long timeout = timeoutAt - System.currentTimeMillis(); + if (!queue.offer(holder, timeout, TimeUnit.MILLISECONDS)) { + throw new RuntimeException(new TimeoutException()); + } + } + } + + private class ValueHolder + { + @Nullable + private final T val; + + private ValueHolder(@Nullable T val) + { + this.val = val; + } + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java index 69fe1909970d..a9316632ca13 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java @@ -105,4 +105,9 @@ default Sequence withBaggage(Closeable baggage) { return Sequences.withBaggage(this, baggage); } + + default Yielder each() + { + return Yielders.each(this); + } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/WrappingSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/WrappingSequence.java index 2fe43bfa6785..f679550f6cb3 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/WrappingSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/WrappingSequence.java @@ -37,7 +37,7 @@ final class WrappingSequence implements Sequence } @Override - public OutType accumulate(final OutType outType, final Accumulator accumulator) + public OutType accumulate(final OutType initValue, final Accumulator accumulator) { OutType result; try { @@ -47,7 +47,7 @@ public OutType accumulate(final OutType outType, final Accumulator Yielder toYielder( final OutType initValue, - final YieldingAccumulator accumulator + YieldingAccumulator accumulator ) { try { diff --git a/processing/src/main/java/org/apache/druid/query/AbstractPrioritizedCallable.java b/core/src/main/java/org/apache/druid/query/AbstractPrioritizedCallable.java similarity index 100% rename from processing/src/main/java/org/apache/druid/query/AbstractPrioritizedCallable.java rename to core/src/main/java/org/apache/druid/query/AbstractPrioritizedCallable.java diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/core/src/main/java/org/apache/druid/query/DruidProcessingConfig.java similarity index 100% rename from processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java rename to core/src/main/java/org/apache/druid/query/DruidProcessingConfig.java diff --git a/core/src/main/java/org/apache/druid/query/ParallelCombines.java b/core/src/main/java/org/apache/druid/query/ParallelCombines.java new file mode 100644 index 000000000000..0425907fbb71 --- /dev/null +++ b/core/src/main/java/org/apache/druid/query/ParallelCombines.java @@ -0,0 +1,127 @@ +/* + * 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.query; + +import com.google.common.collect.Iterables; +import org.apache.druid.java.util.common.Pair; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Future; +import java.util.function.Function; + +public final class ParallelCombines +{ + /** + * Recursively build a combining tree in a bottom-up manner. Each node of the tree is a task that combines input + * iterators asynchronously. + * + * @param leafChildren all child nodes at the leaf level + * @param leafCombineDegree combining degree for the leaf level + * @param intermediateCombineDegree combining degree for intermediate levels + * @param executorSubmitter function to submit a combining task to the executor + * + * @return a pair of a root of the combining tree and a list of futures of all executed combining tasks + */ + public static Pair> buildCombineTree( + List leafChildren, + int leafCombineDegree, + int intermediateCombineDegree, + Function, Pair> executorSubmitter + ) + { + final Pair, List> rootAndFutures = buildCombineTree( + leafChildren, + leafCombineDegree, + intermediateCombineDegree, + executorSubmitter, + true + ); + return Pair.of( + Iterables.getOnlyElement(rootAndFutures.lhs), + rootAndFutures.rhs + ); + } + + private static Pair, List> buildCombineTree( + List children, + int leafCombineDegree, + int intermediateCombineDegree, + Function, Pair> executorSubmitter, + boolean leaf + ) + { + final int numChildLevelIterators = children.size(); + final List childIteratorsOfNextLevel = new ArrayList<>(); + final List combineFutures = new ArrayList<>(); + final int combineDegree = leaf ? leafCombineDegree : intermediateCombineDegree; + + // The below algorithm creates the combining nodes of the current level. It first checks that the number of children + // to be combined together is 1. If it is, the intermediate combining node for that child is not needed. Instead, it + // can be directly connected to a node of the parent level. Here is an example of generated tree when + // numLeafNodes = 6 and leafCombineDegree = intermediateCombineDegree = 2. See the description of + // MINIMUM_LEAF_COMBINE_DEGREE for more details about leafCombineDegree and intermediateCombineDegree. + // + // o + // / \ + // o \ + // / \ \ + // o o o + // / \ / \ / \ + // o o o o o o + // + // We can expect that the aggregates can be combined as early as possible because the tree is built in a bottom-up + // manner. + + for (int i = 0; i < numChildLevelIterators; i += combineDegree) { + if (i < numChildLevelIterators - 1) { + final List subChildren = children.subList( + i, + Math.min(i + combineDegree, numChildLevelIterators) + ); + final Pair iteratorAndFuture = executorSubmitter.apply(subChildren); + + childIteratorsOfNextLevel.add(iteratorAndFuture.lhs); + combineFutures.add(iteratorAndFuture.rhs); + } else { + // If there remains one child, it can be directly connected to a node of the parent level. + childIteratorsOfNextLevel.add(children.get(i)); + } + } + + if (childIteratorsOfNextLevel.size() == 1) { + // This is the root + return Pair.of(childIteratorsOfNextLevel, combineFutures); + } else { + // Build the parent level iterators + final Pair, List> parentIteratorsAndFutures = + buildCombineTree( + childIteratorsOfNextLevel, + leafCombineDegree, + intermediateCombineDegree, + executorSubmitter, + false + ); + combineFutures.addAll(parentIteratorsAndFutures.rhs); + return Pair.of(parentIteratorsAndFutures.lhs, combineFutures); + } + } + + private ParallelCombines() {} +} diff --git a/processing/src/main/java/org/apache/druid/query/PrioritizedCallable.java b/core/src/main/java/org/apache/druid/query/PrioritizedCallable.java similarity index 100% rename from processing/src/main/java/org/apache/druid/query/PrioritizedCallable.java rename to core/src/main/java/org/apache/druid/query/PrioritizedCallable.java diff --git a/processing/src/main/java/org/apache/druid/query/PrioritizedExecutorService.java b/core/src/main/java/org/apache/druid/query/PrioritizedExecutorService.java similarity index 100% rename from processing/src/main/java/org/apache/druid/query/PrioritizedExecutorService.java rename to core/src/main/java/org/apache/druid/query/PrioritizedExecutorService.java diff --git a/processing/src/main/java/org/apache/druid/query/PrioritizedRunnable.java b/core/src/main/java/org/apache/druid/query/PrioritizedRunnable.java similarity index 100% rename from processing/src/main/java/org/apache/druid/query/PrioritizedRunnable.java rename to core/src/main/java/org/apache/druid/query/PrioritizedRunnable.java diff --git a/core/src/main/java/org/apache/druid/query/ThreadResource.java b/core/src/main/java/org/apache/druid/query/ThreadResource.java new file mode 100644 index 000000000000..24ce55763420 --- /dev/null +++ b/core/src/main/java/org/apache/druid/query/ThreadResource.java @@ -0,0 +1,27 @@ +/* + * 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.query; + +/** + * This class can be used to represent a single thread resource when multiple callers contend with each other to get + * thread resources. See org.apache.druid.client.ProcessingThreadResourcePool as an example. + */ +public class ThreadResource +{ +} diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java b/core/src/main/java/org/apache/druid/segment/column/ColumnConfig.java similarity index 100% rename from processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java rename to core/src/main/java/org/apache/druid/segment/column/ColumnConfig.java diff --git a/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java b/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java index 19a21f2c089e..be0242187be7 100644 --- a/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java +++ b/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java @@ -86,9 +86,9 @@ public void testTake() { final ReferenceCountingResourceHolder holder = pool.take(100); assertNotNull(holder); - assertEquals(9, pool.getPoolSize()); + assertEquals(9, pool.available()); holder.close(); - assertEquals(10, pool.getPoolSize()); + assertEquals(10, pool.available()); } @Test(timeout = 60_000L) @@ -106,9 +106,9 @@ public void testTakeBatch() final List> holder = pool.takeBatch(6, 100L); assertNotNull(holder); assertEquals(6, holder.size()); - assertEquals(4, pool.getPoolSize()); + assertEquals(4, pool.available()); holder.forEach(ReferenceCountingResourceHolder::close); - assertEquals(10, pool.getPoolSize()); + assertEquals(10, pool.available()); } @Test(timeout = 60_000L) @@ -117,7 +117,7 @@ public void testWaitAndTakeBatch() throws InterruptedException, ExecutionExcepti List> batchHolder = pool.takeBatch(10, 10); assertNotNull(batchHolder); assertEquals(10, batchHolder.size()); - assertEquals(0, pool.getPoolSize()); + assertEquals(0, pool.available()); final Future>> future = service.submit( () -> pool.takeBatch(8, 100) @@ -128,10 +128,10 @@ public void testWaitAndTakeBatch() throws InterruptedException, ExecutionExcepti batchHolder = future.get(); assertNotNull(batchHolder); assertEquals(8, batchHolder.size()); - assertEquals(2, pool.getPoolSize()); + assertEquals(2, pool.available()); batchHolder.forEach(ReferenceCountingResourceHolder::close); - assertEquals(10, pool.getPoolSize()); + assertEquals(10, pool.available()); } @Test(timeout = 60_000L) @@ -179,7 +179,7 @@ public List> call() final List> r1 = f1.get(); final List> r2 = f2.get(); - assertEquals(0, pool.getPoolSize()); + assertEquals(0, pool.available()); assertTrue(r1.contains(null) || r2.contains(null)); int nonNullCount = 0; @@ -224,7 +224,7 @@ public void run() future1.get(); future2.get(); - assertEquals(pool.maxSize(), pool.getPoolSize()); + assertEquals(pool.maxSize(), pool.available()); } @Test(timeout = 60_000L) @@ -244,17 +244,17 @@ public void testConcurrentTakeBatch() throws ExecutionException, InterruptedExce if (r1 != null) { assertTrue(r2.isEmpty()); - assertEquals(pool.maxSize() - batch1, pool.getPoolSize()); + assertEquals(pool.maxSize() - batch1, pool.available()); assertEquals(batch1, r1.size()); r1.forEach(ReferenceCountingResourceHolder::close); } else { assertNotNull(r2); - assertEquals(pool.maxSize() - batch2, pool.getPoolSize()); + assertEquals(pool.maxSize() - batch2, pool.available()); assertEquals(batch2, r2.size()); r2.forEach(ReferenceCountingResourceHolder::close); } - assertEquals(pool.maxSize(), pool.getPoolSize()); + assertEquals(pool.maxSize(), pool.available()); } @Test(timeout = 60_000L) @@ -276,7 +276,7 @@ public void testConcurrentBatchClose() throws ExecutionException, InterruptedExc assertNotNull(r2); assertEquals(batch1, r1.size()); assertEquals(batch2, r2.size()); - assertEquals(0, pool.getPoolSize()); + assertEquals(0, pool.available()); final Future future1 = service.submit(new Runnable() { @@ -298,7 +298,7 @@ public void run() future1.get(); future2.get(); - assertEquals(pool.maxSize(), pool.getPoolSize()); + assertEquals(pool.maxSize(), pool.available()); } @Test(timeout = 60_000L) @@ -328,9 +328,9 @@ public void run() f1.get(); assertNotNull(r2); assertEquals(10, r2.size()); - assertEquals(0, pool.getPoolSize()); + assertEquals(0, pool.available()); r2.forEach(ReferenceCountingResourceHolder::close); - assertEquals(pool.maxSize(), pool.getPoolSize()); + assertEquals(pool.maxSize(), pool.available()); } } diff --git a/core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java b/core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java deleted file mode 100644 index 741ac7d6e2cb..000000000000 --- a/core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java +++ /dev/null @@ -1,350 +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.collections; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.guava.BaseSequence; -import org.apache.druid.java.util.common.guava.MergeSequence; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.SequenceTestHelper; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.TestSequence; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; - -/** - */ -public class OrderedMergeSequenceTest -{ - @Test - public void testSanity() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(1, 3, 5, 7, 9), - TestSequence.create(2, 8), - TestSequence.create(4, 6, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testMergeEmptySequence() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(ImmutableList.of()), - TestSequence.create(2, 8), - TestSequence.create(4, 6, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testMergeEmptySequenceAtEnd() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(2, 8), - TestSequence.create(4, 6, 8), - TestSequence.create(ImmutableList.of()) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - - @Test - public void testMergeEmptySequenceMiddle() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(2, 8), - TestSequence.create(ImmutableList.of()), - TestSequence.create(4, 6, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testScrewsUpOnOutOfOrderBeginningOfList() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(1, 3, 5, 7, 9), - TestSequence.create(4, 6, 8), - TestSequence.create(2, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(1, 3, 4, 2, 5, 6, 7, 8, 8, 9)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testScrewsUpOnOutOfOrderInList() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(1, 3, 5, 4, 7, 9), - TestSequence.create(2, 8), - TestSequence.create(4, 6) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 4, 6, 7, 8, 9)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testLazinessAccumulation() - { - final ArrayList> sequences = makeSyncedSequences(); - OrderedMergeSequence seq = new OrderedMergeSequence( - Ordering.natural(), Sequences.simple(sequences) - ); - SequenceTestHelper.testAccumulation("", seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)); - } - - @Test - public void testLazinessYielder() throws Exception - { - final ArrayList> sequences = makeSyncedSequences(); - OrderedMergeSequence seq = new OrderedMergeSequence( - Ordering.natural(), Sequences.simple(sequences) - ); - SequenceTestHelper.testYield("", seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)); - } - - private ArrayList> makeSyncedSequences() - { - final boolean[] done = new boolean[]{false, false}; - - final ArrayList> sequences = new ArrayList<>(); - sequences.add( - new BaseSequence>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return Arrays.asList(1, 2, 3).iterator(); - } - - @Override - public void cleanup(Iterator iterFromMake) - { - done[0] = true; - } - } - ) - ); - sequences.add( - new BaseSequence>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return new IteratorShell(Arrays.asList(4, 5, 6).iterator()) - { - int count = 0; - - @Override - public boolean hasNext() - { - if (count >= 1) { - Assert.assertTrue("First iterator not complete", done[0]); - } - return super.hasNext(); - } - - @Override - public Integer next() - { - if (count >= 1) { - Assert.assertTrue("First iterator not complete", done[0]); - } - ++count; - return super.next(); - } - }; - } - - @Override - public void cleanup(Iterator iterFromMake) - { - done[1] = true; - } - } - ) - ); - sequences.add( - new BaseSequence>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return new IteratorShell(Arrays.asList(7, 8, 9).iterator()) - { - int count = 0; - - @Override - public boolean hasNext() - { - if (count >= 1) { - Assert.assertTrue("Second iterator not complete", done[1]); - } - Assert.assertTrue("First iterator not complete", done[0]); - return super.hasNext(); - } - - @Override - public Integer next() - { - if (count >= 1) { - Assert.assertTrue("Second iterator not complete", done[1]); - } - Assert.assertTrue("First iterator not complete", done[0]); - ++count; - return super.next(); - } - }; - } - - @Override - public void cleanup(Iterator iterFromMake) - { - } - } - ) - ); - return sequences; - } - - private OrderedMergeSequence makeMergedSequence( - Ordering ordering, - List> seqs - ) - { - return new OrderedMergeSequence( - ordering, - Sequences.simple((List>) (List) seqs) - ); - } - - private MergeSequence makeUnorderedMergedSequence( - Ordering ordering, - List> seqs - ) - { - return new MergeSequence(ordering, Sequences.simple(seqs)); - } - - @Test - public void testHierarchicalMerge() throws Exception - { - final Sequence seq1 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - - - final Sequence seq2 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - final OrderedMergeSequence finalMerged = new OrderedMergeSequence( - Ordering.natural(), - Sequences.simple( - Lists.newArrayList(seq1, seq2) - ) - ); - - SequenceTestHelper.testAll(finalMerged, Arrays.asList(1, 1)); - } - - @Test - public void testMergeMerge() throws Exception - { - final Sequence seq1 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - - final OrderedMergeSequence finalMerged = new OrderedMergeSequence( - Ordering.natural(), - Sequences.simple( - Collections.singletonList(seq1) - ) - ); - - SequenceTestHelper.testAll(finalMerged, Collections.singletonList(1)); - } - - @Test - public void testOne() throws Exception - { - final MergeSequence seq1 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - - SequenceTestHelper.testAll(seq1, Collections.singletonList(1)); - } -} diff --git a/core/src/test/java/org/apache/druid/common/utils/JodaUtilsTest.java b/core/src/test/java/org/apache/druid/common/utils/JodaUtilsTest.java index c5610f969c80..c9faab573f5b 100644 --- a/core/src/test/java/org/apache/druid/common/utils/JodaUtilsTest.java +++ b/core/src/test/java/org/apache/druid/common/utils/JodaUtilsTest.java @@ -143,5 +143,4 @@ public void testMinMaxPeriod() final Period period = Intervals.ETERNITY.toDuration().toPeriod(); Assert.assertEquals(Long.MAX_VALUE, period.getMinutes()); } - } diff --git a/core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/CombiningSequenceTest.java similarity index 96% rename from core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java rename to core/src/test/java/org/apache/druid/java/util/common/guava/CombiningSequenceTest.java index 9f64d6732863..985010a8a4a0 100644 --- a/core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/CombiningSequenceTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.common.guava; +package org.apache.druid.java.util.common.guava; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; @@ -25,10 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.Pair; -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.YieldingAccumulator; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ComplexSequenceTest.java similarity index 89% rename from core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java rename to core/src/test/java/org/apache/druid/java/util/common/guava/ComplexSequenceTest.java index 74879b339885..57b53698b485 100644 --- a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ComplexSequenceTest.java @@ -17,14 +17,9 @@ * under the License. */ -package org.apache.druid.common.guava; +package org.apache.druid.java.util.common.guava; import com.google.common.primitives.Ints; -import org.apache.druid.java.util.common.guava.Comparators; -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.YieldingAccumulator; import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.junit.Assert; import org.junit.Test; diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java index 61af63869487..228474240836 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; @@ -161,9 +161,8 @@ public void testHierarchicalMerge() throws Exception public void testMergeOne() throws Exception { final Sequence mergeOne = new MergeSequence<>( - Ordering.natural(), Sequences.simple( - Collections.singletonList(TestSequence.create(1)) - ) + Ordering.natural(), + Sequences.simple(Collections.singletonList(TestSequence.create(1))) ); SequenceTestHelper.testAll(mergeOne, Collections.singletonList(1)); diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java new file mode 100644 index 000000000000..9848043df0ab --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java @@ -0,0 +1,171 @@ +/* + * 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.collections.BlockingPool; +import org.apache.druid.collections.DefaultBlockingPool; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.nary.BinaryFn; +import org.apache.druid.query.ThreadResource; +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.Comparator; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ExecutorService; + +public class ParallelMergeCombineSequenceTest +{ + private Random random = new Random(System.currentTimeMillis()); + private ExecutorService service; + private BlockingPool resourcePool; + + @Before + public void setup() + { + service = Execs.multiThreaded(2, "parallel-merge-combine-sequence-test"); + resourcePool = new DefaultBlockingPool<>(ThreadResource::new, 2); + } + + @After + public void teardown() + { + service.shutdown(); + } + + @Test(timeout = 5000L) + public void testSimple() throws InterruptedException + { + 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> input = new ArrayList<>(); + input.add(Sequences.simple(pairs1)); + input.add(Sequences.simple(pairs2)); + input.add(Sequences.simple(pairs3)); + + assertResult(input); + } + + @Test(timeout = 5000L) + public void teatRandom() throws InterruptedException + { + final List> sequences = new ArrayList<>(); + for (int i = 0; i < 7; i++) { + final List pairs = new ArrayList<>(); + final int size = 10000 + random.nextInt(1000); + for (int j = 0; j < size; j++) { + pairs.add(new IntPair(random.nextInt(10000), random.nextInt(500))); + } + pairs.sort(Comparator.comparing(pair -> pair.lhs)); + sequences.add(Sequences.simple(pairs)); + } + + assertResult(sequences); + } + + private void assertResult(List> sequences) throws InterruptedException + { + final Ordering ordering = Ordering.natural().onResultOf(p -> p.lhs); + final BinaryFn 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 ParallelMergeCombineSequence parallelMergeCombineSequence = new ParallelMergeCombineSequence<>( + service, + sequences, + ordering, + mergeFn, + resourcePool.takeBatch(2), + 10240, + true, + 1000, + 0 + ); + + Yielder combiningYielder = Yielders.each(combiningSequence); + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + + IntPair prev = null; + + while (!combiningYielder.isDone() && !parallelMergeCombineYielder.isDone()) { + Assert.assertEquals(combiningYielder.get(), 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 (resourcePool.available() < 2) { + Thread.sleep(100); + } + Assert.assertEquals(2, resourcePool.available()); + } + + private static class IntPair extends Pair + { + IntPair(@Nullable Integer lhs, @Nullable Integer rhs) + { + super(lhs, rhs); + } + } +} diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index 0612a67e4ea9..7a1bf7a82715 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -42,6 +42,8 @@ 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 node| |serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between broker and compute node| +|numBrokerParallelCombineThreads|`0`|Number of threads participating in the parallel combine for this query. To turn on parallel combine, set this to a positive number or -1. If it's -1, the broker uses all current available processing threads.| +|brokerParallelCombineQueueSize|`10240`|Queue size to temporarily keep prefetched and aggregated values in the parallel combine. Note that each thread needs a separate queue| In addition, some query types offer context parameters specific to that query type. 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 a19221807092..cf7ffc6b8784 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -37,6 +37,8 @@ public class QueryContexts public static final String DEFAULT_TIMEOUT_KEY = "defaultTimeout"; @Deprecated public static final String CHUNK_PERIOD_KEY = "chunkPeriod"; + public static final String NUM_BROKER_PARALLEL_COMBINE_THREADS = "numBrokerParallelCombineThreads"; + public static final String BROKER_PARALLEL_COMBINE_QUEUE_SIZE = "brokerParallelCombineQueueSize"; public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; @@ -45,8 +47,11 @@ public class QueryContexts public static final boolean DEFAULT_USE_RESULTLEVEL_CACHE = true; public static final int DEFAULT_PRIORITY = 0; public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0; + public static final int DEFAULT_BROKER_PARALLEL_COMBINE_QUEUE_SIZE = 10240; public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); public static final long NO_TIMEOUT = 0; + public static final int NO_PARALLEL_COMBINE_THREADS = 0; + public static final int NUM_CURRENT_AVAILABLE_THREADS = -1; public static boolean isBySegment(Query query) { @@ -174,6 +179,35 @@ public static Query verifyMaxQueryTimeout(Query query, long maxQueryTi } } + private static int checkPositive(String propertyName, int val) + { + Preconditions.checkArgument( + val > 0, + "%s should be positive, but [%s]", + propertyName, + val + ); + return val; + } + + /** + * Return the configured number of combine threads if any. Others {@link #NO_PARALLEL_COMBINE_THREADS}. + */ + public static int getNumBrokerParallelCombineThreads(Query query) + { + return parseInt(query, NUM_BROKER_PARALLEL_COMBINE_THREADS, NO_PARALLEL_COMBINE_THREADS); + } + + public static int getBrokerParallelCombineQueueSize(Query query) + { + final int queueSize = parseInt( + query, + BROKER_PARALLEL_COMBINE_QUEUE_SIZE, + DEFAULT_BROKER_PARALLEL_COMBINE_QUEUE_SIZE + ); + return checkPositive(BROKER_PARALLEL_COMBINE_QUEUE_SIZE, queueSize); + } + public static long getMaxQueuedBytes(Query query, long defaultValue) { return parseLong(query, MAX_QUEUED_BYTES_KEY, defaultValue); 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 0b890c061331..45a8c51470bb 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.timeline.LogicalSegment; @@ -45,6 +47,16 @@ public abstract class QueryToolChest mergeResults(QueryRunner runner); + /** + * Creates a merge function that is used to merge intermediate aggregates from historicals in broker. This merge + * function is basically used in {@link ResultMergeQueryRunner} and can be used in + * {@link org.apache.druid.java.util.common.guava.ParallelMergeCombineSequence} for parallel merging. + */ + public BinaryFn createMergeFn(QueryType query) + { + throw new UOE("%s doesn't support merge function", query.getClass().getCanonicalName()); + } + /** * Creates a {@link QueryMetrics} object that is used to generate metrics for this specific query type. This exists * to allow for query-specific dimensions and metrics. That is, the ToolChest is expected to set some 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 444aadffb6c2..1b01e75635e1 100644 --- a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java @@ -20,33 +20,37 @@ package org.apache.druid.query; import com.google.common.collect.Ordering; -import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.guava.CombiningSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.nary.BinaryFn; import java.util.Map; +import java.util.function.Function; /** */ @PublicApi public abstract class ResultMergeQueryRunner extends BySegmentSkippingQueryRunner { - public ResultMergeQueryRunner( - QueryRunner baseRunner - ) + private final Function, BinaryFn> mergeFnGenerator; + + public ResultMergeQueryRunner(QueryRunner baseRunner, Function, BinaryFn> mergeFnGenerator) { super(baseRunner); + this.mergeFnGenerator = mergeFnGenerator; } @Override public Sequence doRun(QueryRunner baseRunner, QueryPlus queryPlus, Map context) { Query query = queryPlus.getQuery(); - return CombiningSequence.create(baseRunner.run(queryPlus, context), makeOrdering(query), createMergeFn(query)); + return CombiningSequence.create( + baseRunner.run(queryPlus, context), + makeOrdering(query), + mergeFnGenerator.apply(query) + ); } protected abstract Ordering makeOrdering(Query query); - - protected abstract BinaryFn createMergeFn(Query query); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index a49b48ae216a..edb0cbae9c01 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.common.guava.MappedSequence; 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.nary.BinaryFn; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DataSource; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -132,6 +133,12 @@ public Sequence run(QueryPlus queryPlus, Map responseC }; } + @Override + public BinaryFn createMergeFn(GroupByQuery query) + { + return strategySelector.strategize(query).createMergeFn(query); + } + private Sequence initAndMergeGroupByResults( final GroupByQuery query, QueryRunner runner, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java index 0ef98c6913d7..cbd4109c3b75 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -23,7 +23,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; @@ -36,6 +35,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.AbstractPrioritizedCallable; +import org.apache.druid.query.ParallelCombines; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; @@ -50,7 +50,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.concurrent.CancellationException; @@ -132,7 +131,7 @@ public ParallelCombiner( * @return an iterator of the root grouper of the combining tree */ public CloseableIterator> combine( - List>> sortedIterators, + List>> sortedIterators, List mergedDictionary ) { @@ -160,15 +159,20 @@ public CloseableIterator> combine( final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); - final Pair>>, List> combineIteratorAndFutures = buildCombineTree( - sortedIterators, - bufferSupplier, - combiningFactories, - leafCombineDegree, - mergedDictionary - ); + final Pair>, List> combineIteratorAndFutures = ParallelCombines + .buildCombineTree( + sortedIterators, + leafCombineDegree, + intermediateCombineDegree, + subIterators -> runCombiner( + subIterators, + bufferSupplier.get(), + combiningFactories, + mergedDictionary + ) + ); - final CloseableIterator> combineIterator = Iterables.getOnlyElement(combineIteratorAndFutures.lhs); + final CloseableIterator> combineIterator = combineIteratorAndFutures.lhs; final List combineFutures = combineIteratorAndFutures.rhs; closer.register(() -> checkCombineFutures(combineFutures)); @@ -277,7 +281,7 @@ private Pair findLeafCombineDegreeAndNumBuffers( * * @return minimum number of buffers required for combining tree * - * @see #buildCombineTree + * @see ParallelCombines#buildCombineTree */ private int computeRequiredBufferNum(int numChildNodes, int combineDegree) { @@ -295,87 +299,6 @@ private int computeRequiredBufferNum(int numChildNodes, int combineDegree) } } - /** - * Recursively build a combining tree in a bottom-up manner. Each node of the tree is a task that combines input - * iterators asynchronously. - * - * @param childIterators all iterators of the child level - * @param bufferSupplier combining buffer supplier - * @param combiningFactories array of combining aggregator factories - * @param combineDegree combining degree for the current level - * @param dictionary merged dictionary - * - * @return a pair of a list of iterators of the current level in the combining tree and a list of futures of all - * executed combining tasks - */ - private Pair>>, List> buildCombineTree( - List>> childIterators, - Supplier bufferSupplier, - AggregatorFactory[] combiningFactories, - int combineDegree, - List dictionary - ) - { - final int numChildLevelIterators = childIterators.size(); - final List>> childIteratorsOfNextLevel = new ArrayList<>(); - final List combineFutures = new ArrayList<>(); - - // The below algorithm creates the combining nodes of the current level. It first checks that the number of children - // to be combined together is 1. If it is, the intermediate combining node for that child is not needed. Instead, it - // can be directly connected to a node of the parent level. Here is an example of generated tree when - // numLeafNodes = 6 and leafCombineDegree = intermediateCombineDegree = 2. See the description of - // MINIMUM_LEAF_COMBINE_DEGREE for more details about leafCombineDegree and intermediateCombineDegree. - // - // o - // / \ - // o \ - // / \ \ - // o o o - // / \ / \ / \ - // o o o o o o - // - // We can expect that the aggregates can be combined as early as possible because the tree is built in a bottom-up - // manner. - - for (int i = 0; i < numChildLevelIterators; i += combineDegree) { - if (i < numChildLevelIterators - 1) { - final List>> subIterators = childIterators.subList( - i, - Math.min(i + combineDegree, numChildLevelIterators) - ); - final Pair>, Future> iteratorAndFuture = runCombiner( - subIterators, - bufferSupplier.get(), - combiningFactories, - dictionary - ); - - childIteratorsOfNextLevel.add(iteratorAndFuture.lhs); - combineFutures.add(iteratorAndFuture.rhs); - } else { - // If there remains one child, it can be directly connected to a node of the parent level. - childIteratorsOfNextLevel.add(childIterators.get(i)); - } - } - - if (childIteratorsOfNextLevel.size() == 1) { - // This is the root - return Pair.of(childIteratorsOfNextLevel, combineFutures); - } else { - // Build the parent level iterators - final Pair>>, List> parentIteratorsAndFutures = - buildCombineTree( - childIteratorsOfNextLevel, - bufferSupplier, - combiningFactories, - intermediateCombineDegree, - dictionary - ); - combineFutures.addAll(parentIteratorsAndFutures.rhs); - return Pair.of(parentIteratorsAndFutures.lhs, combineFutures); - } - } - private Pair>, Future> runCombiner( List>> iterators, ByteBuffer combineBuffer, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/TopNSequence.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/TopNSequence.java index bada6d5eea44..e830165a9e27 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/TopNSequence.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/TopNSequence.java @@ -21,7 +21,6 @@ import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; @@ -47,21 +46,14 @@ public Iterator make() } // Materialize the topN values - final MinMaxPriorityQueue queue = MinMaxPriorityQueue - .orderedBy(ordering) - .maximumSize(limit) - .create(); - - input.accumulate( - queue, - new Accumulator, T>() - { - @Override - public MinMaxPriorityQueue accumulate(MinMaxPriorityQueue theQueue, T row) - { - theQueue.offer(row); - return theQueue; - } + final MinMaxPriorityQueue queue = input.accumulate( + MinMaxPriorityQueue + .orderedBy(ordering) + .maximumSize(limit) + .create(), + (theQueue, row) -> { + theQueue.offer(row); + return theQueue; } ); 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 eec4d6f6d6d7..3abf94a672e5 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 @@ -22,7 +22,9 @@ import com.google.common.util.concurrent.ListeningExecutorService; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQuery; @@ -30,6 +32,7 @@ import org.apache.druid.query.groupby.resource.GroupByQueryResource; import org.apache.druid.segment.StorageAdapter; +import javax.annotation.Nullable; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -67,6 +70,15 @@ QueryRunner createIntervalChunkingRunner( GroupByQueryQueryToolChest toolChest ); + /** + * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)}. + */ + @Nullable + default BinaryFn createMergeFn(Query query) + { + return null; + } + Sequence mergeResults(QueryRunner baseRunner, GroupByQuery query, Map responseContext); Sequence applyPostProcessing(Sequence results, GroupByQuery query); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java index 434a828bafe0..059a9cdab758 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -213,6 +213,12 @@ public QueryRunner createIntervalChunkingRunner( return runner; } + @Override + public BinaryFn createMergeFn(Query query) + { + return new GroupByBinaryFnV2((GroupByQuery) query); + } + @Override public Sequence mergeResults( final QueryRunner baseRunner, @@ -222,19 +228,16 @@ public Sequence mergeResults( { // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may // involve materialization) - final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner(baseRunner) + final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner( + baseRunner, + this::createMergeFn + ) { @Override protected Ordering makeOrdering(Query queryParam) { return ((GroupByQuery) queryParam).getRowOrdering(true); } - - @Override - protected BinaryFn createMergeFn(Query queryParam) - { - return new GroupByBinaryFnV2((GroupByQuery) queryParam); - } }; // Fudge timestamp, maybe. 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 4dc50f56887f..97bb9601df0c 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 @@ -30,10 +30,10 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; -import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.CombiningSequence; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.MappedSequence; import org.apache.druid.java.util.common.guava.Sequence; @@ -113,7 +113,8 @@ public Sequence doRun( Map context ) { - SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(config); + SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()) + .withFinalizedAnalysisTypes(config); QueryPlus updatedQueryPlus = queryPlus.withQuery(updatedQuery); return new MappedSequence<>( CombiningSequence.create( diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index 61f77e356962..477aa20012a4 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -99,28 +99,27 @@ public QueryRunner> mergeResults( QueryRunner> runner ) { - return new ResultMergeQueryRunner>(runner) + return new ResultMergeQueryRunner>(runner, q -> createMergeFn((SearchQuery) q)) { @Override protected Ordering> makeOrdering(Query> query) { return ResultGranularTimestampComparator.create( - ((SearchQuery) query).getGranularity(), + query.getGranularity(), query.isDescending() ); } - - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - SearchQuery query = (SearchQuery) input; - return new SearchBinaryFn(query.getSort(), query.getGranularity(), query.getLimit()); - } }; } + @Override + public BinaryFn, Result, Result> createMergeFn( + SearchQuery query + ) + { + return new SearchBinaryFn(query.getSort(), query.getGranularity(), query.getLimit()); + } + @Override public SearchQueryMetrics makeMetrics(SearchQuery query) { diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java index aeaf7fb44bee..a7c0ad452890 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java @@ -109,31 +109,30 @@ public QueryRunner> mergeResults( QueryRunner> queryRunner ) { - return new ResultMergeQueryRunner>(queryRunner) + return new ResultMergeQueryRunner>(queryRunner, q -> createMergeFn((SelectQuery) q)) { @Override protected Ordering> makeOrdering(Query> query) { return ResultGranularTimestampComparator.create( - ((SelectQuery) query).getGranularity(), query.isDescending() - ); - } - - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - SelectQuery query = (SelectQuery) input; - return new SelectBinaryFn( - query.getGranularity(), - query.getPagingSpec(), - query.isDescending() + query.getGranularity(), query.isDescending() ); } }; } + @Override + public BinaryFn, Result, Result> createMergeFn( + SelectQuery query + ) + { + return new SelectBinaryFn( + query.getGranularity(), + query.getPagingSpec(), + query.isDescending() + ); + } + @Override public SelectQueryMetrics makeMetrics(SelectQuery query) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 479fbf0c2d1b..12ffda9bcdee 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -99,7 +99,9 @@ public QueryRunner> mergeResults( ) { final QueryRunner> resultMergeQueryRunner = new ResultMergeQueryRunner>( - queryRunner) + queryRunner, + q -> createMergeFn((TimeseriesQuery) q) + ) { @Override public Sequence> doRun( @@ -125,19 +127,7 @@ public Sequence> doRun( protected Ordering> makeOrdering(Query> query) { return ResultGranularTimestampComparator.create( - ((TimeseriesQuery) query).getGranularity(), query.isDescending() - ); - } - - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - TimeseriesQuery query = (TimeseriesQuery) input; - return new TimeseriesBinaryFn( - query.getGranularity(), - query.getAggregatorSpecs() + query.getGranularity(), query.isDescending() ); } }; @@ -211,6 +201,17 @@ protected BinaryFn, Result, }; } + @Override + public BinaryFn, Result, Result> createMergeFn( + TimeseriesQuery query + ) + { + return new TimeseriesBinaryFn( + query.getGranularity(), + query.getAggregatorSpecs() + ); + } + private Result getNullTimeseriesResultValue(TimeseriesQuery query) { List aggregatorSpecs = query.getAggregatorSpecs(); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java index e140a20453e9..a127a3c65b70 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java @@ -246,6 +246,12 @@ public TopNQueryBuilder granularity(Granularity g) return this; } + public TopNQueryBuilder aggregators(AggregatorFactory... aggregators) + { + aggregatorSpecs = Lists.newArrayList(aggregators); + return this; + } + @SuppressWarnings("unchecked") public TopNQueryBuilder aggregators(List a) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 4774c5c9b1a2..bda00f5a77f9 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -113,34 +113,33 @@ public QueryRunner> mergeResults( QueryRunner> runner ) { - return new ResultMergeQueryRunner>(runner) + return new ResultMergeQueryRunner>(runner, q -> createMergeFn((TopNQuery) q)) { @Override protected Ordering> makeOrdering(Query> query) { return ResultGranularTimestampComparator.create( - ((TopNQuery) query).getGranularity(), query.isDescending() - ); - } - - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - TopNQuery query = (TopNQuery) input; - return new TopNBinaryFn( - query.getGranularity(), - query.getDimensionSpec(), - query.getTopNMetricSpec(), - query.getThreshold(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs() + query.getGranularity(), query.isDescending() ); } }; } + @Override + public BinaryFn, Result, Result> createMergeFn( + TopNQuery query + ) + { + return new TopNBinaryFn( + query.getGranularity(), + query.getDimensionSpec(), + query.getTopNMetricSpec(), + query.getThreshold(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ); + } + @Override public TopNQueryMetrics makeMetrics(TopNQuery query) { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index b013195f6b3a..2d7d6b961e79 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -74,7 +74,7 @@ public TestBlockingPool(Supplier generator, int limit) public ReferenceCountingResourceHolder take(final long timeout) { final ReferenceCountingResourceHolder holder = super.take(timeout); - final int poolSize = getPoolSize(); + final int poolSize = available(); if (minRemainBufferNum > poolSize) { minRemainBufferNum = poolSize; } @@ -85,7 +85,7 @@ public ReferenceCountingResourceHolder take(final long timeout) public List> takeBatch(final int maxElements, final long timeout) { final List> holder = super.takeBatch(maxElements, timeout); - final int poolSize = getPoolSize(); + final int poolSize = available(); if (minRemainBufferNum > poolSize) { minRemainBufferNum = poolSize; } @@ -245,7 +245,7 @@ public void testSimpleGroupBy() GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); assertEquals(2, mergeBufferPool.getMinRemainBufferNum()); - assertEquals(3, mergeBufferPool.getPoolSize()); + assertEquals(3, mergeBufferPool.available()); } @Test @@ -273,7 +273,7 @@ public void testNestedGroupBy() GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); assertEquals(1, mergeBufferPool.getMinRemainBufferNum()); - assertEquals(3, mergeBufferPool.getPoolSize()); + assertEquals(3, mergeBufferPool.available()); } @Test @@ -313,7 +313,7 @@ public void testDoubleNestedGroupBy() // This should be 0 because the broker needs 2 buffers and the queryable node needs one. assertEquals(0, mergeBufferPool.getMinRemainBufferNum()); - assertEquals(3, mergeBufferPool.getPoolSize()); + assertEquals(3, mergeBufferPool.available()); } @Test @@ -366,6 +366,6 @@ public void testTripleNestedGroupBy() // This should be 0 because the broker needs 2 buffers and the queryable node needs one. assertEquals(0, mergeBufferPool.getMinRemainBufferNum()); - assertEquals(3, mergeBufferPool.getPoolSize()); + assertEquals(3, mergeBufferPool.available()); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index 8f542c3ff233..d3af79755ae3 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -109,7 +109,7 @@ public void testCombine() throws IOException } final int leafNum = 8; - final List iterators = new ArrayList<>(leafNum); + final List>> iterators = new ArrayList<>(leafNum); for (int i = 0; i < leafNum; i++) { iterators.add(new TestIterator(baseIterator.iterator())); } @@ -121,6 +121,6 @@ public void testCombine() throws IOException } } - iterators.forEach(it -> Assert.assertTrue(it.isClosed())); + iterators.forEach(it -> Assert.assertTrue(((TestIterator) it).isClosed())); } } diff --git a/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 930e32175031..c7eb40ac755f 100644 --- a/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -68,7 +68,7 @@ public void testRetry() throws Exception new QueryRunner() { @Override - public Sequence run(QueryPlus queryPlus, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return new Sequence() { diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 574cdc874b2f..747c97e4683c 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -24,7 +24,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; -import org.apache.druid.client.selector.QueryableDruidServer; +import org.apache.druid.client.selector.RemoteDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.client.selector.TierSelectorStrategy; import org.apache.druid.guice.annotations.EscalatedClient; @@ -62,7 +62,7 @@ public class BrokerServerView implements TimelineServerView private final Object lock = new Object(); - private final ConcurrentMap clients; + private final ConcurrentMap clients; private final Map selectors; private final Map> timelines; private final ConcurrentMap timelineCallbacks = new ConcurrentHashMap<>(); @@ -170,10 +170,10 @@ public boolean isInitialized() return initialized; } - private QueryableDruidServer addServer(DruidServer server) + private RemoteDruidServer addServer(DruidServer server) { - QueryableDruidServer retVal = new QueryableDruidServer(server, makeDirectClient(server)); - QueryableDruidServer exists = clients.put(server.getName(), retVal); + RemoteDruidServer retVal = new RemoteDruidServer(server, makeDirectClient(server)); + RemoteDruidServer exists = clients.put(server.getName(), retVal); if (exists != null) { log.warn("QueryRunner for server[%s] already existed!? Well it's getting replaced", server); } @@ -186,7 +186,7 @@ private DirectDruidClient makeDirectClient(DruidServer server) return new DirectDruidClient(warehouse, queryWatcher, smileMapper, httpClient, server.getScheme(), server.getHost(), emitter); } - private QueryableDruidServer removeServer(DruidServer server) + private RemoteDruidServer removeServer(DruidServer server) { for (DataSegment segment : server.getSegments().values()) { serverRemovedSegment(server.getMetadata(), segment); @@ -216,7 +216,7 @@ private void serverAddedSegment(final DruidServerMetadata server, final DataSegm selectors.put(segmentId, selector); } - QueryableDruidServer queryableDruidServer = clients.get(server.getName()); + RemoteDruidServer queryableDruidServer = clients.get(server.getName()); if (queryableDruidServer == null) { queryableDruidServer = addServer(baseView.getInventoryValue(server.getName())); } @@ -240,7 +240,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen return; } - QueryableDruidServer queryableDruidServer = clients.get(server.getName()); + RemoteDruidServer queryableDruidServer = clients.get(server.getName()); if (!selector.removeServer(queryableDruidServer)) { log.warn( "Asked to disassociate non-existant association between server[%s] and segment[%s]", @@ -291,12 +291,12 @@ public void registerTimelineCallback(final Executor exec, final TimelineCallback public QueryRunner getQueryRunner(DruidServer server) { synchronized (lock) { - QueryableDruidServer queryableDruidServer = clients.get(server.getName()); + RemoteDruidServer queryableDruidServer = clients.get(server.getName()); if (queryableDruidServer == null) { log.error("WTF?! No QueryableDruidServer found for %s", server.getName()); return null; } - return queryableDruidServer.getClient(); + return queryableDruidServer.getQueryRunner(); } } 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 ad40a06b84fe..d99338985447 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -32,25 +32,33 @@ import com.google.common.hash.Hashing; import com.google.inject.Inject; import org.apache.commons.codec.binary.Base64; +import org.apache.druid.client.ProcessingThreadResourcePool.ReserveResult; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; 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.collections.ReferenceCountingResourceHolder; import org.apache.druid.guice.annotations.Client; +import org.apache.druid.guice.annotations.Processing; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; 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.CombiningSequence; import org.apache.druid.java.util.common.guava.LazySequence; +import org.apache.druid.java.util.common.guava.ParallelMergeCombineSequence; 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.nary.BinaryFn; 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; @@ -60,6 +68,7 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.ThreadResource; import org.apache.druid.query.aggregation.MetricManipulatorFns; import org.apache.druid.query.filter.DimFilterUtils; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; @@ -86,6 +95,7 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; +import java.util.concurrent.ExecutorService; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -101,6 +111,8 @@ public class CachingClusteredClient implements QuerySegmentWalker private final CachePopulator cachePopulator; private final CacheConfig cacheConfig; private final DruidHttpClientConfig httpClientConfig; + private final ExecutorService processingPool; + private final ProcessingThreadResourcePool processingThreadResourcePool; @Inject public CachingClusteredClient( @@ -110,7 +122,9 @@ public CachingClusteredClient( @Smile ObjectMapper objectMapper, CachePopulator cachePopulator, CacheConfig cacheConfig, - @Client DruidHttpClientConfig httpClientConfig + @Client DruidHttpClientConfig httpClientConfig, + @Processing ExecutorService processingPool, + DruidProcessingConfig processingConfig ) { this.warehouse = warehouse; @@ -120,6 +134,8 @@ public CachingClusteredClient( this.cachePopulator = cachePopulator; this.cacheConfig = cacheConfig; this.httpClientConfig = httpClientConfig; + this.processingPool = processingPool; + this.processingThreadResourcePool = new ProcessingThreadResourcePool(processingConfig.getNumThreads()); if (cacheConfig.isQueryCacheable(Query.GROUP_BY) && (cacheConfig.isUseCache() || cacheConfig.isPopulateCache())) { log.warn( @@ -285,12 +301,72 @@ Sequence run(final UnaryOperator> time List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); addSequencesFromServer(sequencesByInterval, segmentsByServer); - return Sequences - .simple(sequencesByInterval) - .flatMerge(seq -> seq, query.getResultOrdering()); + return merge(sequencesByInterval); }); } + private Sequence merge(List> sequencesByInterval) + { + final int numParallelCombineThreads = QueryContexts.getNumBrokerParallelCombineThreads(query); + + if (numParallelCombineThreads > 0) { + final ReserveResult reserveResult = processingThreadResourcePool.reserve(query, numParallelCombineThreads); + if (!reserveResult.isOk()) { + throw new ISE( + "Not enough processing threads. The query needs [%d] threads, but only [%d] were available", + numParallelCombineThreads, + reserveResult.getNumAvailableResources() + ); + } + return parallelMerge(sequencesByInterval, reserveResult.getResources()); + } else if (numParallelCombineThreads == QueryContexts.NUM_CURRENT_AVAILABLE_THREADS) { + final ReserveResult reserveResult = processingThreadResourcePool.reserve(query, numParallelCombineThreads); + if (reserveResult.isOk()) { + return parallelMerge(sequencesByInterval, reserveResult.getResources()); + } else { + return sequentialMerge(sequencesByInterval); + } + } else if (numParallelCombineThreads == QueryContexts.NO_PARALLEL_COMBINE_THREADS) { + return sequentialMerge(sequencesByInterval); + } else { + throw new ISE( + "Unknown value[%d] for [%s]", + numParallelCombineThreads, + QueryContexts.NUM_BROKER_PARALLEL_COMBINE_THREADS + ); + } + } + + private Sequence parallelMerge( + List> sequencesByInterval, + List> threadResources + ) + { + final BinaryFn mergeFn = toolChest.createMergeFn(query); + return CombiningSequence.create( + new ParallelMergeCombineSequence<>( + processingPool, + sequencesByInterval, + query.getResultOrdering(), + mergeFn, + threadResources, + QueryContexts.getBrokerParallelCombineQueueSize(query), + QueryContexts.hasTimeout(query), + QueryContexts.getTimeout(query), + QueryContexts.getPriority(query) + ), + query.getResultOrdering(), + mergeFn + ); + } + + private Sequence sequentialMerge(List> sequencesByInterval) + { + return Sequences + .simple(sequencesByInterval) + .flatMerge(seq -> seq, query.getResultOrdering()); + } + private Set computeSegmentsToQuery(TimelineLookup timeline) { final List> serversLookup = toolChest.filterSegments( diff --git a/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java b/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java new file mode 100644 index 000000000000..eee82215d894 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java @@ -0,0 +1,86 @@ +/* + * 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.client; + +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.DefaultBlockingPool; +import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.ThreadResource; + +import java.util.List; + +/** + * A pool to coordinate callers which contend with each other to get thread resources. This class just provides a + * logical coordination and the real thread pool should be maintained separately. + * This can be improved to consider query priority in the future. + */ +class ProcessingThreadResourcePool +{ + private final BlockingPool resourcePool; + + ProcessingThreadResourcePool(int poolSize) + { + this.resourcePool = new DefaultBlockingPool<>(ThreadResource::new, poolSize); + } + + ReserveResult reserve(Query query, int n) + { + final boolean hasTimeout = QueryContexts.hasTimeout(query); + final long timeout = QueryContexts.getTimeout(query); + + if (n == QueryContexts.NUM_CURRENT_AVAILABLE_THREADS) { + final List> availableResources = resourcePool.pollAll(); + return new ReserveResult(availableResources, availableResources.size()); + } else { + return new ReserveResult( + hasTimeout ? resourcePool.takeBatch(n, timeout) : resourcePool.takeBatch(n), + resourcePool.available() + ); + } + } + + static class ReserveResult + { + private final List> resources; + private final int numAvailableResources; + + private ReserveResult(List> resources, int numAvailableResources) + { + this.resources = resources; + this.numAvailableResources = numAvailableResources; + } + + boolean isOk() + { + return !resources.isEmpty(); + } + + List> getResources() + { + return resources; + } + + int getNumAvailableResources() + { + return numAvailableResources; + } + } +} diff --git a/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java b/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java new file mode 100644 index 000000000000..63482a434a4c --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.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.client; + +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.BySegmentQueryRunner; +import org.apache.druid.query.FinalizeResultsQueryRunner; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactory; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexSegment; + +import java.util.Map; + +public class SimpleQueryRunner implements QueryRunner +{ + private final QueryRunnerFactoryConglomerate conglomerate; + private final QueryableIndexSegment segment; + + public SimpleQueryRunner(QueryRunnerFactoryConglomerate conglomerate, String segmentId, QueryableIndex queryableIndex) + { + this.conglomerate = conglomerate; + this.segment = new QueryableIndexSegment(segmentId, queryableIndex); + } + + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + final QueryRunnerFactory factory = conglomerate.findFactory(queryPlus.getQuery()); + //noinspection unchecked + return factory.getToolchest().preMergeQueryDecoration( + new FinalizeResultsQueryRunner<>( + new BySegmentQueryRunner<>( + segment.getIdentifier(), + segment.getDataInterval().getStart(), + factory.createRunner(segment) + ), + factory.getToolchest() + ) + ).run(queryPlus, responseContext); + } +} diff --git a/server/src/main/java/org/apache/druid/client/cache/Cache.java b/server/src/main/java/org/apache/druid/client/cache/Cache.java index 725a880a4da7..43c549b39ab6 100644 --- a/server/src/main/java/org/apache/druid/client/cache/Cache.java +++ b/server/src/main/java/org/apache/druid/client/cache/Cache.java @@ -34,12 +34,14 @@ public interface Cache { @Nullable byte[] get(NamedKey key); + void put(NamedKey key, byte[] value); /** * Resulting map should not contain any null values (i.e. cache misses should not be included) * * @param keys + * * @return */ Map getBulk(Iterable keys); @@ -52,6 +54,7 @@ public interface Cache /** * Custom metrics not covered by CacheStats may be emitted by this method. + * * @param emitter The service emitter to emit on. */ void doMonitor(ServiceEmitter emitter); @@ -73,9 +76,9 @@ public byte[] toByteArray() { final byte[] nsBytes = StringUtils.toUtf8(this.namespace); return ByteBuffer.allocate(Integer.BYTES + nsBytes.length + this.key.length) - .putInt(nsBytes.length) - .put(nsBytes) - .put(this.key).array(); + .putInt(nsBytes.length) + .put(nsBytes) + .put(this.key).array(); } @Override diff --git a/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java b/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java index fc401c424e3c..598fca288cdb 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java +++ b/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java @@ -27,16 +27,20 @@ import java.util.Comparator; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; public class ConnectionCountServerSelectorStrategy implements ServerSelectorStrategy { - private static final Comparator COMPARATOR = - Comparator.comparingInt(s -> s.getClient().getNumOpenConnections()); + private static final Comparator COMPARATOR = + Comparator.comparingInt(s -> s.getQueryRunner().getNumOpenConnections()); @Override public QueryableDruidServer pick(Set servers, DataSegment segment) { - return Collections.min(servers, COMPARATOR); + return Collections.min( + servers.stream().map(server -> (RemoteDruidServer) server).collect(Collectors.toSet()), + COMPARATOR + ); } @Override @@ -45,6 +49,14 @@ public List pick(Set servers, DataSe if (servers.size() <= numServersToPick) { return ImmutableList.copyOf(servers); } - return Ordering.from(COMPARATOR).leastOf(servers, numServersToPick); + return Ordering + .from(COMPARATOR) + .leastOf( + servers.stream().map(server -> (RemoteDruidServer) server).collect(Collectors.toSet()), + numServersToPick + ) + .stream() + .map(server -> (QueryableDruidServer) server) + .collect(Collectors.toList()); } } diff --git a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java index b98f1518b147..e154f889bda0 100644 --- a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java +++ b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java @@ -16,41 +16,18 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.druid.client.selector; -import org.apache.druid.client.DirectDruidClient; import org.apache.druid.client.DruidServer; +import org.apache.druid.query.QueryRunner; /** + * Interface to represent a single server which can participate in query processing. The main implementation is + * {@link RemoteDruidServer}. */ -public class QueryableDruidServer +public interface QueryableDruidServer { - private final DruidServer server; - private final DirectDruidClient client; - - public QueryableDruidServer(DruidServer server, DirectDruidClient client) - { - this.server = server; - this.client = client; - } - - public DruidServer getServer() - { - return server; - } - - public DirectDruidClient getClient() - { - return client; - } + DruidServer getServer(); - @Override - public String toString() - { - return "QueryableDruidServer{" + - "server=" + server + - ", client=" + client + - '}'; - } + T getQueryRunner(); } diff --git a/server/src/main/java/org/apache/druid/client/selector/RemoteDruidServer.java b/server/src/main/java/org/apache/druid/client/selector/RemoteDruidServer.java new file mode 100644 index 000000000000..40abeca14550 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/selector/RemoteDruidServer.java @@ -0,0 +1,58 @@ +/* + * 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.client.selector; + +import org.apache.druid.client.DirectDruidClient; +import org.apache.druid.client.DruidServer; + +/** + */ +public class RemoteDruidServer implements QueryableDruidServer +{ + private final DruidServer server; + private final DirectDruidClient client; + + public RemoteDruidServer(DruidServer server, DirectDruidClient client) + { + this.server = server; + this.client = client; + } + + @Override + public DruidServer getServer() + { + return server; + } + + @Override + public DirectDruidClient getQueryRunner() + { + return client; + } + + @Override + public String toString() + { + return "RemoteDruidServer{" + + "server=" + server + + ", client=" + client + + '}'; + } +} 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 47407215af0b..f5d34acf1f62 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -32,14 +32,18 @@ import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.client.cache.MapCache; import org.apache.druid.client.selector.QueryableDruidServer; +import org.apache.druid.client.selector.RemoteDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.client.selector.TierSelectorStrategy; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; 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.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; @@ -98,6 +102,21 @@ public void setUp() { timeline = new VersionedIntervalTimeline<>(Ordering.natural()); serverView = EasyMock.createNiceMock(TimelineServerView.class); + + final QueryRunner emptyQueryRunner = EasyMock.createStrictMock(QueryRunner.class); + + EasyMock.expect( + emptyQueryRunner.run(EasyMock.anyObject(), EasyMock.anyObject()) + ).andReturn( + Sequences.empty() + ).anyTimes(); + EasyMock.expect( + serverView.getQueryRunner(EasyMock.anyObject()) + ).andReturn( + emptyQueryRunner + ).anyTimes(); + + EasyMock.replay(serverView, emptyQueryRunner); cache = MapCache.create(100000); client = makeClient( new ForegroundCachePopulator(OBJECT_MAPPER, new CachePopulatorStats(), -1) @@ -202,7 +221,7 @@ public QueryableDruidServer pick( DataSegment segment ) { - return new QueryableDruidServer( + return new RemoteDruidServer( new DruidServer("localhost", "localhost", null, 100, ServerType.HISTORICAL, "a", 10), EasyMock.createNiceMock(DirectDruidClient.class) ); @@ -216,7 +235,7 @@ public List pick( ) { return Collections.singletonList( - new QueryableDruidServer( + new RemoteDruidServer( new DruidServer("localhost", "localhost", null, 100, ServerType.HISTORICAL, "a", 10), EasyMock.createNiceMock(DirectDruidClient.class) ) @@ -307,12 +326,28 @@ public int getCacheBulkMergeLimit() return mergeLimit; } }, - new DruidHttpClientConfig() { + new DruidHttpClientConfig() + { @Override public long getMaxQueuedBytes() { return 0L; } + }, + Execs.multiThreaded(2, "caching-clustered-client-functionality-test"), + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int getNumThreads() + { + return 2; + } } ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java new file mode 100644 index 000000000000..a41e99b0a657 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java @@ -0,0 +1,109 @@ +/* + * 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.client; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.VirtualColumns; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +public class CachingClusteredClientParallelMergeGroupByTest extends CachingClusteredClientParallelMergeTestBase +{ + private static final int DIM1_CARD = 11; + private static final int DIM2_CARD = 3; + + @Override + void addServers(TestTimelineServerView serverView, int numServers, Random random) + { + for (int i = 0; i < numServers; i++) { + final int numRows = random.nextInt(10000) + 10000; + final List rows = new ArrayList<>(numRows); + for (int j = 0; j < numRows; j++) { + final Row row = createRow( + "2018-01-01", + StringUtils.format("dim1_%d", random.nextInt(DIM1_CARD)), + StringUtils.format("dim2_%d", random.nextInt(DIM2_CARD)), + random.nextInt(100), + random.nextDouble() + ); + rows.add(row); + } + rows.sort( + (r1, r2) -> { + final int cmp = r1.getDimension("dim1").get(0).compareTo(r2.getDimension("dim1").get(0)); + if (cmp == 0) { + return r1.getDimension("dim2").get(0).compareTo(r2.getDimension("dim2").get(0)); + } else { + return cmp; + } + } + ); + serverView.addServer(createServer(i + 1), new TestQueryRunner<>(rows)); + } + } + + private static Row createRow(String timestamp, String dim1, String dim2, long cnt, double doubleMet) + { + return new MapBasedRow( + DateTimes.of(timestamp), + ImmutableMap.of("dim1", dim1, "dim2", dim2, "cnt", cnt, "double_max", doubleMet) + ); + } + + @Test + public void test() + { + final GroupByQuery expectedQuery = new GroupByQuery( + new TableDataSource(DATA_SOURCE), + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2018-01-01/2018-01-07"))), + VirtualColumns.EMPTY, + null, + Granularities.ALL, + ImmutableList.of(new DefaultDimensionSpec("dim1", "dim1"), new DefaultDimensionSpec("dim2", "dim2")), + ImmutableList.of( + new LongSumAggregatorFactory("cnt", "cnt"), + new DoubleMaxAggregatorFactory("double_max", "double_met") + ), + null, + null, + null, + null, + null + ); + + runAndVerify(expectedQuery); + } +} diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java new file mode 100644 index 000000000000..d567dd7c94cc --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java @@ -0,0 +1,268 @@ +/* + * 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.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.ForegroundCachePopulator; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.collections.CloseableStupidPool; +import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +import org.apache.druid.query.DruidProcessingConfig; +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; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryConfig; +import org.apache.druid.query.topn.TopNQueryQueryToolChest; +import org.apache.druid.query.topn.TopNQueryRunnerFactory; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.Random; +import java.util.concurrent.ExecutorService; + +public abstract class CachingClusteredClientParallelMergeTestBase +{ + static final String DATA_SOURCE = "test"; + + private static final String VERSION = "version"; + private static final int NUM_SERVERS = 5; + private static final int NUM_THREADS = 2; + + private final Random random = new Random(System.currentTimeMillis()); + + private ExecutorService executorService; + private CachingClusteredClient client; + private QueryToolChestWarehouse toolChestWarehouse; + + @Before + public void setup() + { + final DruidProcessingConfig processingConfig = new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + // Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby. + // Two buffers for the broker and one for the queryable + return 3; + } + + @Override + public int getNumThreads() + { + return NUM_THREADS; + } + }; + + final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>builder() + .put( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TopNQuery.class, + new TopNQueryRunnerFactory( + new CloseableStupidPool<>( + "TopNQueryRunnerFactory-bufferPool", + () -> ByteBuffer.allocate(10 * 1024 * 1024) + ), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + GroupByQuery.class, + GroupByQueryRunnerTest + .makeQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + }, + processingConfig + ).lhs + ) + .build() + ); + + toolChestWarehouse = new QueryToolChestWarehouse() + { + @Override + public > QueryToolChest getToolChest(final QueryType query) + { + return conglomerate.findFactory(query).getToolchest(); + } + }; + + final TestTimelineServerView serverView = new TestTimelineServerView(); + + addServers(serverView, NUM_SERVERS, random); + + serverView.addSegmentToServer( + createServer(1), + createSegment("2018-01-01/2018-01-07", 0) + ); + serverView.addSegmentToServer( + createServer(2), + createSegment("2018-01-01/2018-01-07", 1) + ); + serverView.addSegmentToServer( + createServer(3), + createSegment("2018-01-01/2018-01-07", 2) + ); + + final ObjectMapper objectMapper = new DefaultObjectMapper(); + executorService = Execs.multiThreaded( + processingConfig.getNumThreads(), + "caching-clustered-client-parallel-merge-test" + ); + client = new CachingClusteredClient( + toolChestWarehouse, + serverView, + MapCache.create(1024), + objectMapper, + new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 1024), + new CacheConfig(), + new DruidHttpClientConfig(), + executorService, + processingConfig + ); + } + + @After + public void tearDown() + { + executorService.shutdown(); + } + + abstract void addServers(TestTimelineServerView serverView, int numServers, Random random); + + static DruidServer createServer(int nameSuiffix) + { + return new DruidServer( + "server_" + nameSuiffix, + "127.0.0." + nameSuiffix, + null, + 10240L, + ServerType.HISTORICAL, + "default", + 0 + ); + } + + private static DataSegment createSegment(String interval, int partitionNum) + { + return new DataSegment( + DATA_SOURCE, + Intervals.of(interval), + VERSION, + Collections.emptyMap(), + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("cnt", "double_met"), + new NumberedShardSpec(partitionNum, 0), + 0, + 1024L + ); + } + + void runAndVerify(Query expectedQuery) + { + final Query testQuery = expectedQuery.withOverriddenContext( + ImmutableMap.of(QueryContexts.NUM_BROKER_PARALLEL_COMBINE_THREADS, QueryContexts.NUM_CURRENT_AVAILABLE_THREADS) + ); + final Sequence result = runQuery(testQuery); + final Sequence expected = runQuery(expectedQuery); + + Assert.assertEquals(expected.toList(), result.toList()); + } + + private Sequence runQuery(Query query) + { + final QueryRunner queryRunner = client.getQueryRunnerForIntervals( + query, + Collections.singletonList(Intervals.of("2018-01-01/2018-01-07")) + ); + + //noinspection unchecked + return new FluentQueryRunnerBuilder<>(toolChestWarehouse.getToolChest(query)) + .create(queryRunner) + .applyPreMergeDecoration() + .mergeResults() + .applyPostMergeDecoration() + .run(QueryPlus.wrap(query), new HashMap<>()); + } +} diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java new file mode 100644 index 000000000000..419c233c2e54 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java @@ -0,0 +1,96 @@ +/* + * 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.client; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.Result; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.DateTime; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +public class CachingClusteredClientParallelMergeTimeseriesTest extends CachingClusteredClientParallelMergeTestBase +{ + @Override + void addServers(TestTimelineServerView serverView, int numServers, Random random) + { + for (int i = 0; i < numServers; i++) { + final List> totalRows = new ArrayList<>(); + + for (int day = 1; day < 6; day++) { + for (int hour = 1; hour < 24; hour++) { + for (int minute = 1; minute < 60; minute++) { + final Result row = createRow( + DateTimes.of("2018-01-%02dT%02d:%02d:00", day, hour, minute), + random.nextInt(100), + random.nextDouble() + ); + totalRows.add(row); + } + } + } + + serverView.addServer(createServer(i + 1), new TestQueryRunner<>(totalRows)); + } + } + + private static Result createRow(DateTime timestamp, long cnt, double doubleMet) + { + return new Result<>( + timestamp, + new TimeseriesResultValue(ImmutableMap.of("cnt", cnt, "double_max", doubleMet)) + ); + } + + @Test + public void test() + { + final TimeseriesQuery expectedQuery = new TimeseriesQuery( + new TableDataSource(DATA_SOURCE), + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2018-01-01/2018-01-07"))), + false, + VirtualColumns.EMPTY, + null, + Granularities.MINUTE, + ImmutableList.of( + new LongSumAggregatorFactory("cnt", "cnt"), + new DoubleMaxAggregatorFactory("double_max", "double_met") + ), + null, + 0, + null + ); + + runAndVerify(expectedQuery); + } +} diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java new file mode 100644 index 000000000000..0321b1b47c6c --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java @@ -0,0 +1,100 @@ +/* + * 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.client; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.Result; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.topn.NumericTopNMetricSpec; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNResultValue; +import org.apache.druid.segment.VirtualColumns; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Random; + +public class CachingClusteredClientParallelMergeTopNTest extends CachingClusteredClientParallelMergeTestBase +{ + @Override + void addServers(TestTimelineServerView serverView, int numServers, Random random) + { + for (int i = 0; i < numServers; i++) { + final List> totalRows = new ArrayList<>(); + + for (int day = 1; day < 6; day++) { + final int numRows = random.nextInt(10000) + 10000; + final List> rows = new ArrayList<>(numRows); + for (int j = 0; j < numRows; j++) { + final Map row = createRow( + StringUtils.format("dim2_%d", j), + random.nextInt(100), + random.nextDouble() + ); + rows.add(row); + } + rows.sort(Comparator.comparing(row -> (String) row.get("dim2"))); + totalRows.add(new Result<>(DateTimes.of("2018-01-0%d", day), new TopNResultValue(rows))); + } + + serverView.addServer(createServer(i + 1), new TestQueryRunner<>(totalRows)); + } + } + + private static Map createRow(String dim2, long cnt, double doubleMet) + { + return ImmutableMap.of("dim2", dim2, "cnt", cnt, "double_max", doubleMet); + } + + @Test + public void test() + { + final TopNQuery expectedQuery = new TopNQuery( + new TableDataSource(DATA_SOURCE), + VirtualColumns.EMPTY, + new DefaultDimensionSpec("dim2", "dim2"), + new NumericTopNMetricSpec("double_max"), + 20000, // should be larger than cardinalrity of dim2 + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2018-01-01/2018-01-07"))), + null, + Granularities.DAY, + ImmutableList.of( + new LongSumAggregatorFactory("cnt", "cnt"), + new DoubleMaxAggregatorFactory("double_max", "double_met") + ), + null, + null + ); + + runAndVerify(expectedQuery); + } +} 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 f65dc3070d8c..382f023dc9aa 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -50,8 +50,8 @@ import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.client.cache.MapCache; import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; -import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.RandomServerSelectorStrategy; +import org.apache.druid.client.selector.RemoteDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -62,6 +62,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -74,6 +75,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; @@ -280,14 +282,7 @@ public static Iterable constructorFeeder() { return Lists.transform( Lists.newArrayList(new RangeIterable(RANDOMNESS)), - new Function() - { - @Override - public Object[] apply(Integer input) - { - return new Object[]{input}; - } - } + input -> new Object[]{input} ); } @@ -569,7 +564,7 @@ public void testCachingOverBulkLimitEnforcesLimit() dataSegment, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) ); - selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), dataSegment); + selector.addServerAndUpdateSegment(new RemoteDruidServer(lastServer, null), dataSegment); timeline.add(interval, "v", new SingleElementPartitionChunk<>(selector)); getDefaultQueryRunner().run(QueryPlus.wrap(query), context); @@ -586,9 +581,11 @@ public void testCachingOverBulkLimitEnforcesLimit() .once(); EasyMock.replay(cache); client = makeClient(new ForegroundCachePopulator(JSON_MAPPER, new CachePopulatorStats(), -1), cache, 0); + getDefaultQueryRunner().run(QueryPlus.wrap(query), context); EasyMock.verify(cache); EasyMock.verify(dataSegment); + Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured()); Assert.assertTrue("Cache Keys empty", ImmutableList.copyOf(cacheKeyCapture.getValue()).isEmpty()); } @@ -1174,14 +1171,14 @@ public void testSearchCaching() public void testSearchCachingRenamedOutput() { final Druids.SearchQueryBuilder builder = Druids.newSearchQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .limit(1000) - .intervals(SEG_SPEC) - .dimensions(Collections.singletonList(TOP_DIM)) - .query("how") - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .limit(1000) + .intervals(SEG_SPEC) + .dimensions(Collections.singletonList(TOP_DIM)) + .query("how") + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1347,14 +1344,14 @@ public void testSelectCachingRenamedOutputName() final Set metrics = Sets.newHashSet("rows"); Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .dimensions(Collections.singletonList("a")) - .metrics(Collections.singletonList("rows")) - .pagingSpec(new PagingSpec(null, 3)) - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .dimensions(Collections.singletonList("a")) + .metrics(Collections.singletonList("rows")) + .pagingSpec(new PagingSpec(null, 3)) + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1446,9 +1443,9 @@ public void testSelectCachingRenamedOutputName() public void testGroupByCaching() { List aggsWithUniques = ImmutableList.builder() - .addAll(AGGS) - .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) - .build(); + .addAll(AGGS) + .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) + .build(); final HashFunction hashFn = Hashing.murmur3_128(); @@ -1650,9 +1647,11 @@ For dim1 (2011-01-06/2011-01-10), the combined range for the bound filters is {( makeTimeResults(DateTimes.of("2011-01-01"), 50, 5000, DateTimes.of("2011-01-02"), 10, 1252, DateTimes.of("2011-01-03"), 20, 6213, - DateTimes.of("2011-01-04"), 30, 743), + DateTimes.of("2011-01-04"), 30, 743 + ), makeTimeResults(DateTimes.of("2011-01-07"), 60, 6020, - DateTimes.of("2011-01-08"), 70, 250) + DateTimes.of("2011-01-08"), 70, 250 + ) ); testQueryCachingWithFilter( @@ -1690,14 +1689,14 @@ public void testSingleDimensionPruning() ); final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(filter) - .granularity(GRANULARITY) - .intervals(SEG_SPEC) - .context(CONTEXT) - .intervals("2011-01-05/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS); + .dataSource(DATA_SOURCE) + .filters(filter) + .granularity(GRANULARITY) + .intervals(SEG_SPEC) + .context(CONTEXT) + .intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS); TimeseriesQuery query = builder.build(); Map context = new HashMap<>(); @@ -1770,7 +1769,7 @@ private ServerSelector makeMockSingleDimensionSelector( segment, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) ); - selector.addServerAndUpdateSegment(new QueryableDruidServer(server, null), segment); + selector.addServerAndUpdateSegment(new RemoteDruidServer(server, null), segment); return selector; } @@ -1888,7 +1887,12 @@ public void testQueryCachingWithFilter( @Override public Sequence answer() { - return toFilteredQueryableTimeseriesResults((TimeseriesQuery) capture.getValue().getQuery(), segmentIds, queryIntervals, results); + return toFilteredQueryableTimeseriesResults( + (TimeseriesQuery) capture.getValue().getQuery(), + segmentIds, + queryIntervals, + results + ); } }) .times(0, 1); @@ -1946,7 +1950,11 @@ private Sequence> toFilteredQueryableTimeseriesRes MultipleSpecificSegmentSpec spec = (MultipleSpecificSegmentSpec) query.getQuerySegmentSpec(); List> ret = new ArrayList<>(); for (SegmentDescriptor descriptor : spec.getDescriptors()) { - String id = StringUtils.format("%s_%s", queryIntervals.indexOf(descriptor.getInterval()), descriptor.getPartitionNumber()); + String id = StringUtils.format( + "%s_%s", + queryIntervals.indexOf(descriptor.getInterval()), + descriptor.getPartitionNumber() + ); int index = segmentIds.indexOf(id); if (index != -1) { ret.add(new Result( @@ -2012,8 +2020,8 @@ public void testQueryCaching( .andReturn(expectations.getQueryRunner()) .once(); - final Capture capture = new Capture(); - final Capture context = new Capture(); + final Capture capture = EasyMock.newCapture(); + final Capture context = EasyMock.newCapture(); queryCaptures.add(capture); QueryRunner queryable = expectations.getQueryRunner(); @@ -2106,50 +2114,40 @@ public void testQueryCaching( } runWithMocks( - new Runnable() - { - @Override - public void run() - { - HashMap context = new HashMap(); - for (int i = 0; i < numTimesToQuery; ++i) { - TestHelper.assertExpectedResults( - new MergeIterable<>( - Comparators.naturalNullsFirst(), - FunctionalIterable - .create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd)) - .transformCat( - new Function>>>() - { - @Override - public Iterable>> apply(@Nullable Integer input) - { - List>> retVal = new ArrayList<>(); - - final Map exps = serverExpectationList.get(input); - for (ServerExpectations expectations : exps.values()) { - for (ServerExpectation expectation : expectations) { - retVal.add(expectation.getResults()); - } - } - - return retVal; + () -> { + HashMap context = new HashMap<>(); + for (int i1 = 0; i1 < numTimesToQuery; ++i1) { + TestHelper.assertExpectedResults( + new MergeIterable<>( + Comparators.naturalNullsFirst(), + FunctionalIterable + .create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd)) + .transformCat( + (Function>>>) input -> { + List>> retVal = new ArrayList<>(); + + final Map exps = serverExpectationList.get(input); + for (ServerExpectations expectations : exps.values()) { + for (ServerExpectation expectation : expectations) { + retVal.add(expectation.getResults()); } } - ) - ), - runner.run( - QueryPlus.wrap( - query.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval)) - ) - ), - context - ) - ); - if (queryCompletedCallback != null) { - queryCompletedCallback.run(); - } + + return retVal; + } + ) + ), + runner.run( + QueryPlus.wrap( + query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval)) + ) + ), + context + ) + ); + if (queryCompletedCallback != null) { + queryCompletedCallback.run(); } } }, @@ -2206,7 +2204,7 @@ private List> populateTimeline( expectation.getSegment(), new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) ); - selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), selector.getSegment()); + selector.addServerAndUpdateSegment(new RemoteDruidServer(lastServer, null), selector.getSegment()); final ShardSpec shardSpec; if (numChunks == 1) { @@ -2466,13 +2464,13 @@ private Iterable> makeTimeResults(Object... object (DateTime) objects[i], new TimeseriesResultValue( ImmutableMap.builder() - .put("rows", objects[i + 1]) - .put("imps", objects[i + 2]) - .put("impers", objects[i + 2]) - .put("avg_imps_per_row", avg_impr) - .put("avg_imps_per_row_half", avg_impr / 2) - .put("avg_imps_per_row_double", avg_impr * 2) - .build() + .put("rows", objects[i + 1]) + .put("imps", objects[i + 2]) + .put("impers", objects[i + 2]) + .put("avg_imps_per_row", avg_impr) + .put("avg_imps_per_row_half", avg_impr / 2) + .put("avg_imps_per_row_double", avg_impr * 2) + .build() ) ) ); @@ -2722,12 +2720,28 @@ public int getCacheBulkMergeLimit() return mergeLimit; } }, - new DruidHttpClientConfig() { + new DruidHttpClientConfig() + { @Override public long getMaxQueuedBytes() { return 0L; } + }, + Execs.multiThreaded(2, "parallel-merge-comine-sequence-test"), + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int getNumThreads() + { + return 2; + } } ); } @@ -3107,13 +3121,16 @@ public void testIfNoneMatch() dataSegment, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) ); - selector.addServerAndUpdateSegment(new QueryableDruidServer(servers[0], null), dataSegment); + selector.addServerAndUpdateSegment(new RemoteDruidServer(servers[0], null), dataSegment); timeline.add(interval, "ver", new SingleElementPartitionChunk<>(selector)); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() .dataSource(DATA_SOURCE) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(interval))) - .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) + .context(ImmutableMap.of( + "If-None-Match", + "aVJV29CJY93rszVW/QBy0arWZo0=" + )) .build(); @@ -3126,13 +3143,8 @@ public void testIfNoneMatch() @SuppressWarnings("unchecked") private QueryRunner getDefaultQueryRunner() { - return new QueryRunner() { - @Override - public Sequence run(final QueryPlus queryPlus, final Map responseContext) - { - return client.getQueryRunnerForIntervals(queryPlus.getQuery(), queryPlus.getQuery().getIntervals()) - .run(queryPlus, responseContext); - } - }; + return (queryPlus, responseContext) -> client + .getQueryRunnerForIntervals(queryPlus.getQuery(), queryPlus.getQuery().getIntervals()) + .run(queryPlus, responseContext); } } diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java index 65a8cd62c44f..64c4be20f863 100644 --- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java @@ -25,7 +25,7 @@ import com.google.common.util.concurrent.SettableFuture; import org.apache.druid.client.selector.ConnectionCountServerSelectorStrategy; import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; -import org.apache.druid.client.selector.QueryableDruidServer; +import org.apache.druid.client.selector.RemoteDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -151,12 +151,12 @@ public void testRun() throws Exception new NoopServiceEmitter() ); - QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer( + RemoteDruidServer queryableDruidServer1 = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.REALTIME, DruidServer.DEFAULT_TIER, 0), client1 ); serverSelector.addServerAndUpdateSegment(queryableDruidServer1, serverSelector.getSegment()); - QueryableDruidServer queryableDruidServer2 = new QueryableDruidServer( + RemoteDruidServer queryableDruidServer2 = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client2 ); @@ -260,7 +260,7 @@ public void testCancel() new NoopServiceEmitter() ); - QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer( + RemoteDruidServer queryableDruidServer1 = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client1 ); @@ -331,7 +331,7 @@ public void testQueryInterruptionExceptionLogMessage() new NoopServiceEmitter() ); - QueryableDruidServer queryableDruidServer = new QueryableDruidServer( + RemoteDruidServer queryableDruidServer = new RemoteDruidServer( new DruidServer("test1", hostName, null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client1 ); diff --git a/server/src/test/java/org/apache/druid/client/TestQueryRunner.java b/server/src/test/java/org/apache/druid/client/TestQueryRunner.java new file mode 100644 index 000000000000..e9ff5c8b88f6 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/TestQueryRunner.java @@ -0,0 +1,46 @@ +/* + * 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.client; + +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.logger.Logger; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; + +import java.util.List; +import java.util.Map; + +public class TestQueryRunner implements QueryRunner +{ + private static final Logger log = new Logger(TestQueryRunner.class); + + private final Sequence sequence; + + public TestQueryRunner(List iterable) + { + sequence = Sequences.simple(iterable); + } + + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + return sequence; + } +} diff --git a/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java b/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java new file mode 100644 index 000000000000..1d2a808d1790 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java @@ -0,0 +1,132 @@ +/* + * 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.client; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; +import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; +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.client.selector.TierSelectorStrategy; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +import org.apache.druid.timeline.VersionedIntervalTimeline; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executor; + +public class TestTimelineServerView implements TimelineServerView +{ + private final TierSelectorStrategy tierSelectorStrategy = new HighestPriorityTierSelectorStrategy( + new RandomServerSelectorStrategy() + ); + // server -> queryRunner + private final Map servers = new HashMap<>(); + // segmentId -> serverSelector + private final Map selectors = new HashMap<>(); + // dataSource -> version -> serverSelector + private final Map> timelines = new HashMap<>(); + + void addServer(DruidServer server, TestQueryRunner queryRunner) + { + servers.put(server, new TestDruidServer(server, queryRunner)); + } + + void addSegmentToServer(DruidServer server, DataSegment segment) + { + final ServerSelector selector = selectors.computeIfAbsent( + segment.getIdentifier(), + k -> new ServerSelector(segment, tierSelectorStrategy) + ); + selector.addServerAndUpdateSegment(servers.get(server), segment); + timelines.computeIfAbsent(segment.getDataSource(), k -> new VersionedIntervalTimeline<>(Ordering.natural())) + .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); + } + + @Nullable + @Override + public TimelineLookup getTimeline(DataSource dataSource) + { + final String table = Iterables.getOnlyElement(dataSource.getNames()); + return timelines.get(table); + } + + @Override + public List getDruidServers() + { + return Collections.emptyList(); + } + + @Override + public QueryRunner getQueryRunner(DruidServer server) + { + final TestDruidServer queryableDruidServer = Preconditions.checkNotNull(servers.get(server), "server"); + return (QueryRunner) queryableDruidServer.queryRunner; + } + + @Override + public void registerTimelineCallback(Executor exec, TimelineCallback callback) + { + // do nothing + } + + @Override + public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback callback) + { + // do nothing + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + // do nothing + } + + private static class TestDruidServer implements QueryableDruidServer + { + private final DruidServer server; + private final TestQueryRunner queryRunner; + + TestDruidServer(DruidServer server, TestQueryRunner queryRunner) + { + this.server = server; + this.queryRunner = queryRunner; + } + + @Override + public DruidServer getServer() + { + return server; + } + + @Override + public TestQueryRunner getQueryRunner() + { + return queryRunner; + } + } +} diff --git a/server/src/test/java/org/apache/druid/client/selector/ServerSelectorTest.java b/server/src/test/java/org/apache/druid/client/selector/ServerSelectorTest.java index 290b18470a9f..c039ba14548d 100644 --- a/server/src/test/java/org/apache/druid/client/selector/ServerSelectorTest.java +++ b/server/src/test/java/org/apache/druid/client/selector/ServerSelectorTest.java @@ -73,7 +73,7 @@ public void testSegmentUpdate() ); selector.addServerAndUpdateSegment( - new QueryableDruidServer( + new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), EasyMock.createMock(DirectDruidClient.class) ), diff --git a/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java b/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java index ce7fe8aac86a..9d1f42d53af0 100644 --- a/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java +++ b/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java @@ -44,11 +44,11 @@ public class TierSelectorStrategyTest public void testHighestPriorityTierSelectorStrategy() { DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class); - QueryableDruidServer lowPriority = new QueryableDruidServer( + RemoteDruidServer lowPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client ); - QueryableDruidServer highPriority = new QueryableDruidServer( + RemoteDruidServer highPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), client ); @@ -63,11 +63,11 @@ public void testHighestPriorityTierSelectorStrategy() public void testLowestPriorityTierSelectorStrategy() { DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class); - QueryableDruidServer lowPriority = new QueryableDruidServer( + RemoteDruidServer lowPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client ); - QueryableDruidServer highPriority = new QueryableDruidServer( + RemoteDruidServer highPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), client ); @@ -82,15 +82,15 @@ public void testLowestPriorityTierSelectorStrategy() public void testCustomPriorityTierSelectorStrategy() { DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class); - QueryableDruidServer lowPriority = new QueryableDruidServer( + RemoteDruidServer lowPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, -1), client ); - QueryableDruidServer mediumPriority = new QueryableDruidServer( + RemoteDruidServer mediumPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client ); - QueryableDruidServer highPriority = new QueryableDruidServer( + RemoteDruidServer highPriority = new RemoteDruidServer( new DruidServer("test1", "localhost", null, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), client ); @@ -113,7 +113,7 @@ public List getPriorities() private void testTierSelectorStrategy( TierSelectorStrategy tierSelectorStrategy, - QueryableDruidServer... expectedSelection + RemoteDruidServer... expectedSelection ) { final ServerSelector serverSelector = new ServerSelector( @@ -131,14 +131,14 @@ private void testTierSelectorStrategy( tierSelectorStrategy ); - List servers = new ArrayList<>(Arrays.asList(expectedSelection)); + List servers = new ArrayList<>(Arrays.asList(expectedSelection)); List expectedCandidates = new ArrayList<>(); - for (QueryableDruidServer server : servers) { + for (RemoteDruidServer server : servers) { expectedCandidates.add(server.getServer().getMetadata()); } Collections.shuffle(servers); - for (QueryableDruidServer server : servers) { + for (RemoteDruidServer server : servers) { serverSelector.addServerAndUpdateSegment(server, serverSelector.getSegment()); } From 737d2e63bbfa9324773f3ec099c75a5b2efacf30 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 15 Nov 2018 19:48:18 -0800 Subject: [PATCH 2/5] Fix teamcity --- .../main/java/org/apache/druid/query/topn/TopNQueryBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java index a127a3c65b70..420f08e4213d 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java @@ -248,7 +248,7 @@ public TopNQueryBuilder granularity(Granularity g) public TopNQueryBuilder aggregators(AggregatorFactory... aggregators) { - aggregatorSpecs = Lists.newArrayList(aggregators); + aggregatorSpecs = Arrays.asList(aggregators); return this; } From 97c17b053b780d154f60d8b6747f920d21034120 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 5 Dec 2018 10:14:47 -0800 Subject: [PATCH 3/5] Fix checkstyle --- .../druid/benchmark/query/CachingClusteredClientBenchmark.java | 1 + .../java/util/common/guava/ParallelMergeCombineSequence.java | 1 + core/src/main/java/org/apache/druid/query/ParallelCombines.java | 1 + core/src/main/java/org/apache/druid/query/ThreadResource.java | 1 + .../org/apache/druid/client/ProcessingThreadResourcePool.java | 1 + .../src/main/java/org/apache/druid/client/SimpleQueryRunner.java | 1 + .../org/apache/druid/client/selector/QueryableDruidServer.java | 1 + .../client/CachingClusteredClientParallelMergeGroupByTest.java | 1 + .../client/CachingClusteredClientParallelMergeTestBase.java | 1 + .../CachingClusteredClientParallelMergeTimeseriesTest.java | 1 + .../client/CachingClusteredClientParallelMergeTopNTest.java | 1 + .../src/test/java/org/apache/druid/client/TestQueryRunner.java | 1 + .../java/org/apache/druid/client/TestTimelineServerView.java | 1 + 13 files changed, 13 insertions(+) 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 2e036f5ad220..db9359061250 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 @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.benchmark.query; import com.fasterxml.jackson.databind.InjectableValues.Std; diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java index 97c583f5a95c..e5d189438ecd 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequence.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.java.util.common.guava; import com.google.common.collect.Ordering; diff --git a/core/src/main/java/org/apache/druid/query/ParallelCombines.java b/core/src/main/java/org/apache/druid/query/ParallelCombines.java index 0425907fbb71..fb07adcec203 100644 --- a/core/src/main/java/org/apache/druid/query/ParallelCombines.java +++ b/core/src/main/java/org/apache/druid/query/ParallelCombines.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query; import com.google.common.collect.Iterables; diff --git a/core/src/main/java/org/apache/druid/query/ThreadResource.java b/core/src/main/java/org/apache/druid/query/ThreadResource.java index 24ce55763420..2e47299dac2b 100644 --- a/core/src/main/java/org/apache/druid/query/ThreadResource.java +++ b/core/src/main/java/org/apache/druid/query/ThreadResource.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query; /** diff --git a/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java b/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java index eee82215d894..0043b59f435e 100644 --- a/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java +++ b/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import org.apache.druid.collections.BlockingPool; diff --git a/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java b/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java index 63482a434a4c..16983ff3960a 100644 --- a/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java +++ b/server/src/main/java/org/apache/druid/client/SimpleQueryRunner.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import org.apache.druid.java.util.common.guava.Sequence; diff --git a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java index e154f889bda0..db3d11bc7b21 100644 --- a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java +++ b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client.selector; import org.apache.druid.client.DruidServer; diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java index a41e99b0a657..d88573ef1e4a 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeGroupByTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.google.common.collect.ImmutableList; diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java index d567dd7c94cc..0a1a52718549 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTestBase.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java index 419c233c2e54..788f48d9f763 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTimeseriesTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.google.common.collect.ImmutableList; diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java index 0321b1b47c6c..061c07de9f1a 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientParallelMergeTopNTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.google.common.collect.ImmutableList; diff --git a/server/src/test/java/org/apache/druid/client/TestQueryRunner.java b/server/src/test/java/org/apache/druid/client/TestQueryRunner.java index e9ff5c8b88f6..a3c837c39c23 100644 --- a/server/src/test/java/org/apache/druid/client/TestQueryRunner.java +++ b/server/src/test/java/org/apache/druid/client/TestQueryRunner.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import org.apache.druid.java.util.common.guava.Sequence; diff --git a/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java b/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java index 1d2a808d1790..144237226bc1 100644 --- a/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java +++ b/server/src/test/java/org/apache/druid/client/TestTimelineServerView.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.google.common.base.Preconditions; From 7f709c97187011b4963914e112e0fa3b15bd479a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 5 Dec 2018 11:37:12 -0800 Subject: [PATCH 4/5] add doc --- docs/content/querying/query-context.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index 7a1bf7a82715..3b5332ddc862 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -42,7 +42,7 @@ 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 node| |serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between broker and compute node| -|numBrokerParallelCombineThreads|`0`|Number of threads participating in the parallel combine for this query. To turn on parallel combine, set this to a positive number or -1. If it's -1, the broker uses all current available processing threads.| +|numBrokerParallelCombineThreads|`0`|Number of threads participating in the parallel combine for this query. To turn on parallel combine, set this to a positive number or -1. If it's -1, the broker uses all current available processing threads. This is disabled by default because parallel combine may not be good if the size of intermediate aggregates is not large enough.| |brokerParallelCombineQueueSize|`10240`|Queue size to temporarily keep prefetched and aggregated values in the parallel combine. Note that each thread needs a separate queue| In addition, some query types offer context parameters specific to that query type. From 89d8bb38f2c3888b5dc080489b9b0bcfb3cfda9c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 11 Dec 2018 20:54:59 -0800 Subject: [PATCH 5/5] fix race when calling available() --- .../druid/collections/BlockingPool.java | 42 +++++++++++++++---- .../collections/DefaultBlockingPool.java | 24 +++++------ .../druid/collections/DummyBlockingPool.java | 10 +---- .../druid/collections/BlockingPoolTest.java | 22 +++++----- .../ParallelMergeCombineSequenceTest.java | 5 +-- .../GroupByMergingQueryRunnerV2.java | 8 +++- .../groupby/strategy/GroupByStrategyV2.java | 12 +++--- .../groupby/GroupByQueryMergeBufferTest.java | 6 +-- .../GroupByQueryRunnerFailureTest.java | 9 ++-- .../client/ProcessingThreadResourcePool.java | 8 +++- 10 files changed, 85 insertions(+), 61 deletions(-) diff --git a/core/src/main/java/org/apache/druid/collections/BlockingPool.java b/core/src/main/java/org/apache/druid/collections/BlockingPool.java index 1c698cc93981..ca732cb96cee 100644 --- a/core/src/main/java/org/apache/druid/collections/BlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/BlockingPool.java @@ -29,11 +29,6 @@ public interface BlockingPool */ int maxSize(); - /** - * Returns the number of current available resources. - */ - int available(); - /** * Poll all available resources from the pool. If there's no available resource, it returns an empty list. */ @@ -64,16 +59,45 @@ public interface BlockingPool * @param elementNum number of resources to take * @param timeoutMs maximum time to wait for resources, in milliseconds. * - * @return a list of resource holders. An empty list is returned if {@code elementNum} resources aren't available. + * @return {@link TakeBatchResult} containing a list of resource holders if it succeeds. An empty list should be + * returned if there aren't enough available resources. The result also contains the number of remaining + * resources after this call no matter it succeeded or not. */ - List> takeBatch(int elementNum, long timeoutMs); + TakeBatchResult takeBatch(int elementNum, long timeoutMs); /** * Take resources from the pool, waiting if necessary until the elements of the given number become available. * * @param elementNum number of resources to take * - * @return a list of resource holders. An empty list is returned if {@code elementNum} resources aren't available. + * @return {@link TakeBatchResult} containing a list of resource holders and the number of remaining resources. */ - List> takeBatch(int elementNum); + TakeBatchResult takeBatch(int elementNum); + + class TakeBatchResult + { + private final List> elements; + private final int numAvailableElements; + + public TakeBatchResult(List> elements, int numAvailableElements) + { + this.elements = elements; + this.numAvailableElements = numAvailableElements; + } + + public boolean isOk() + { + return elements.size() > 0; + } + + public List> getElements() + { + return elements; + } + + public int getNumAvailableElements() + { + return numAvailableElements; + } + } } diff --git a/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java index 56a9edbef07e..284e90b10f88 100644 --- a/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java @@ -70,17 +70,10 @@ public int maxSize() return maxSize; } - @Override + @VisibleForTesting public int available() { - final ReentrantLock lock = this.lock; - lock.lock(); - try { - return objects.size(); - } - finally { - lock.unlock(); - } + return objects.size(); } @Override @@ -184,13 +177,14 @@ private T takeObject() throws InterruptedException } @Override - public List> takeBatch(final int elementNum, final long timeoutMs) + public TakeBatchResult takeBatch(final int elementNum, final long timeoutMs) { + Preconditions.checkArgument(elementNum > 0, "elementNum should be positive"); Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs); checkInitialized(); try { final List objects = timeoutMs > 0 ? pollObjects(elementNum, timeoutMs) : pollObjects(elementNum); - return objects.stream().map(this::wrapObject).collect(Collectors.toList()); + return new TakeBatchResult<>(objects.stream().map(this::wrapObject).collect(Collectors.toList()), objects.size()); } catch (InterruptedException e) { throw new RuntimeException(e); @@ -198,11 +192,15 @@ public List> takeBatch(final int elementNum, } @Override - public List> takeBatch(final int elementNum) + public TakeBatchResult takeBatch(final int elementNum) { + Preconditions.checkArgument(elementNum > 0, "elementNum should be positive"); checkInitialized(); try { - return takeObjects(elementNum).stream().map(this::wrapObject).collect(Collectors.toList()); + return new TakeBatchResult<>( + takeObjects(elementNum).stream().map(this::wrapObject).collect(Collectors.toList()), + objects.size() + ); } catch (InterruptedException e) { throw new RuntimeException(e); diff --git a/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java index 1e41858d9d40..60a4b5795f7c 100644 --- a/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java @@ -44,12 +44,6 @@ public int maxSize() return 0; } - @Override - public int available() - { - return 0; - } - @Override public List> pollAll() { @@ -69,13 +63,13 @@ public ReferenceCountingResourceHolder take() } @Override - public List> takeBatch(int elementNum, long timeoutMs) + public TakeBatchResult takeBatch(int elementNum, long timeoutMs) { throw new UnsupportedOperationException(); } @Override - public List> takeBatch(int elementNum) + public TakeBatchResult takeBatch(int elementNum) { throw new UnsupportedOperationException(); } diff --git a/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java b/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java index be0242187be7..8854cada28ba 100644 --- a/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java +++ b/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java @@ -94,7 +94,7 @@ public void testTake() @Test(timeout = 60_000L) public void testTakeTimeout() { - final List> batchHolder = pool.takeBatch(10, 100L); + final List> batchHolder = pool.takeBatch(10, 100L).getElements(); final ReferenceCountingResourceHolder holder = pool.take(100); assertNull(holder); batchHolder.forEach(ReferenceCountingResourceHolder::close); @@ -103,7 +103,7 @@ public void testTakeTimeout() @Test(timeout = 60_000L) public void testTakeBatch() { - final List> holder = pool.takeBatch(6, 100L); + final List> holder = pool.takeBatch(6, 100L).getElements(); assertNotNull(holder); assertEquals(6, holder.size()); assertEquals(4, pool.available()); @@ -114,13 +114,13 @@ public void testTakeBatch() @Test(timeout = 60_000L) public void testWaitAndTakeBatch() throws InterruptedException, ExecutionException { - List> batchHolder = pool.takeBatch(10, 10); + List> batchHolder = pool.takeBatch(10, 10).getElements(); assertNotNull(batchHolder); assertEquals(10, batchHolder.size()); assertEquals(0, pool.available()); final Future>> future = service.submit( - () -> pool.takeBatch(8, 100) + () -> pool.takeBatch(8, 100).getElements() ); Thread.sleep(20); batchHolder.forEach(ReferenceCountingResourceHolder::close); @@ -137,7 +137,7 @@ public void testWaitAndTakeBatch() throws InterruptedException, ExecutionExcepti @Test(timeout = 60_000L) public void testTakeBatchTooManyObjects() { - final List> holder = pool.takeBatch(100, 100L); + final List> holder = pool.takeBatch(100, 100L).getElements(); assertTrue(holder.isEmpty()); } @@ -231,10 +231,10 @@ public void run() public void testConcurrentTakeBatch() throws ExecutionException, InterruptedException { final int batch1 = pool.maxSize() / 2; - final Callable>> c1 = () -> pool.takeBatch(batch1, 10); + final Callable>> c1 = () -> pool.takeBatch(batch1, 10).getElements(); final int batch2 = pool.maxSize() - batch1 + 1; - final Callable>> c2 = () -> pool.takeBatch(batch2, 10); + final Callable>> c2 = () -> pool.takeBatch(batch2, 10).getElements(); final Future>> f1 = service.submit(c1); final Future>> f2 = service.submit(c2); @@ -261,10 +261,10 @@ public void testConcurrentTakeBatch() throws ExecutionException, InterruptedExce public void testConcurrentBatchClose() throws ExecutionException, InterruptedException { final int batch1 = pool.maxSize() / 2; - final Callable>> c1 = () -> pool.takeBatch(batch1, 10); + final Callable>> c1 = () -> pool.takeBatch(batch1, 10).getElements(); final int batch2 = pool.maxSize() - batch1; - final Callable>> c2 = () -> pool.takeBatch(batch2, 10); + final Callable>> c2 = () -> pool.takeBatch(batch2, 10).getElements(); final Future>> f1 = service.submit(c1); final Future>> f2 = service.submit(c2); @@ -304,9 +304,9 @@ public void run() @Test(timeout = 60_000L) public void testConcurrentTakeBatchClose() throws ExecutionException, InterruptedException { - final List> r1 = pool.takeBatch(1, 10); + final List> r1 = pool.takeBatch(1, 10).getElements(); - final Callable>> c2 = () -> pool.takeBatch(10, 100); + final Callable>> c2 = () -> pool.takeBatch(10, 100).getElements(); final Future>> f2 = service.submit(c2); final Future f1 = service.submit(new Runnable() diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java index 9848043df0ab..681e29695e87 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombineSequenceTest.java @@ -20,7 +20,6 @@ package org.apache.druid.java.util.common.guava; import com.google.common.collect.Ordering; -import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; @@ -43,7 +42,7 @@ public class ParallelMergeCombineSequenceTest { private Random random = new Random(System.currentTimeMillis()); private ExecutorService service; - private BlockingPool resourcePool; + private DefaultBlockingPool resourcePool; @Before public void setup() @@ -133,7 +132,7 @@ private void assertResult(List> sequences) throws InterruptedE sequences, ordering, mergeFn, - resourcePool.takeBatch(2), + resourcePool.takeBatch(2).getElements(), 10240, true, 1000, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index d13096cb5e60..e32ef4e96cac 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -33,6 +33,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.BlockingPool.TakeBatchResult; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.Releaser; import org.apache.druid.data.input.Row; @@ -334,11 +335,14 @@ private List> getMergeBuffersHolder( if (timeout <= 0) { throw new TimeoutException(); } - if ((mergeBufferHolder = mergeBufferPool.takeBatch(numBuffers, timeout)).isEmpty()) { + final TakeBatchResult result = mergeBufferPool.takeBatch(numBuffers, timeout); + if (!result.isOk()) { throw new TimeoutException("Cannot acquire enough merge buffers"); + } else { + mergeBufferHolder = result.getElements(); } } else { - mergeBufferHolder = mergeBufferPool.takeBatch(numBuffers); + mergeBufferHolder = mergeBufferPool.takeBatch(numBuffers).getElements(); } return mergeBufferHolder; } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java index 059a9cdab758..2db7935ced63 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -30,8 +30,8 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.inject.Inject; import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.BlockingPool.TakeBatchResult; import org.apache.druid.collections.NonBlockingPool; -import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.guice.annotations.Global; @@ -152,16 +152,16 @@ public GroupByQueryResource prepareResource(GroupByQuery query, boolean willMerg } else if (requiredMergeBufferNum == 0) { return new GroupByQueryResource(); } else { - final List> mergeBufferHolders; + final TakeBatchResult takeBatchResult; if (QueryContexts.hasTimeout(query)) { - mergeBufferHolders = mergeBufferPool.takeBatch(requiredMergeBufferNum, QueryContexts.getTimeout(query)); + takeBatchResult = mergeBufferPool.takeBatch(requiredMergeBufferNum, QueryContexts.getTimeout(query)); } else { - mergeBufferHolders = mergeBufferPool.takeBatch(requiredMergeBufferNum); + takeBatchResult = mergeBufferPool.takeBatch(requiredMergeBufferNum); } - if (mergeBufferHolders.isEmpty()) { + if (!takeBatchResult.isOk()) { throw new InsufficientResourcesException("Cannot acquire enough merge buffers"); } else { - return new GroupByQueryResource(mergeBufferHolders); + return new GroupByQueryResource(takeBatchResult.getElements()); } } } else { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index 6fee899028a4..fbcdd0edbf24 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -83,14 +83,14 @@ public ReferenceCountingResourceHolder take(final long timeout) } @Override - public List> takeBatch(final int maxElements, final long timeout) + public TakeBatchResult takeBatch(final int maxElements, final long timeout) { - final List> holder = super.takeBatch(maxElements, timeout); + final TakeBatchResult result = super.takeBatch(maxElements, timeout); final int poolSize = available(); if (minRemainBufferNum > poolSize) { minRemainBufferNum = poolSize; } - return holder; + return result; } public void resetMinRemainBufferNum() diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index a54a210bc32c..d166c66ee51b 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.collections.BlockingPool.TakeBatchResult; import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; @@ -275,14 +276,14 @@ public void testInsufficientResourcesOnBroker() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500)) .build(); - List> holder = null; + TakeBatchResult result = null; try { - holder = mergeBufferPool.takeBatch(1, 10); + result = mergeBufferPool.takeBatch(1, 10); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } finally { - if (holder != null) { - holder.forEach(ReferenceCountingResourceHolder::close); + if (result != null && result.isOk()) { + result.getElements().forEach(ReferenceCountingResourceHolder::close); } } } diff --git a/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java b/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java index 0043b59f435e..d92f36862216 100644 --- a/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java +++ b/server/src/main/java/org/apache/druid/client/ProcessingThreadResourcePool.java @@ -20,6 +20,7 @@ package org.apache.druid.client; import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.BlockingPool.TakeBatchResult; import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.query.Query; @@ -51,9 +52,12 @@ ReserveResult reserve(Query query, int n) final List> availableResources = resourcePool.pollAll(); return new ReserveResult(availableResources, availableResources.size()); } else { + final TakeBatchResult result = hasTimeout + ? resourcePool.takeBatch(n, timeout) + : resourcePool.takeBatch(n); return new ReserveResult( - hasTimeout ? resourcePool.takeBatch(n, timeout) : resourcePool.takeBatch(n), - resourcePool.available() + result.getElements(), + result.getNumAvailableElements() ); } }